From 07f0ac03fdac133342d907298b092fe0f734967e Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 30 Jun 2017 16:21:09 -0700 Subject: [PATCH 01/34] Extension points for authentication/authorization --- .../druid/benchmark/query/SqlBenchmark.java | 5 +- docs/content/configuration/auth.md | 48 ++ .../AuthenticationKerberosConfig.java | 25 +- .../DruidKerberosAuthenticationHandler.java | 229 +++++++ .../kerberos/DruidKerberosModule.java | 48 +- .../security/kerberos/DruidKerberosUtil.java | 4 +- .../kerberos/KerberosAuthenticator.java | 599 ++++++++++++++++++ .../security/kerberos/SpnegoFilterConfig.java | 10 +- .../security/kerberos/SpnegoFilterHolder.java | 139 ---- .../sql/QuantileSqlAggregatorTest.java | 5 +- .../druid/indexing/kafka/KafkaIndexTask.java | 149 ++++- .../kafka/supervisor/KafkaSupervisor.java | 8 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 14 +- .../indexing/kafka/KafkaIndexTaskTest.java | 3 + .../kafka/supervisor/KafkaSupervisorTest.java | 8 +- .../io.druid.initialization.DruidModule | 2 +- .../overlord/http/OverlordResource.java | 196 +++--- .../security/SupervisorResourceFilter.java | 51 +- .../http/security/TaskResourceFilter.java | 27 +- .../supervisor/SupervisorResource.java | 110 ++-- .../io/druid/indexing/common/TestUtils.java | 4 + .../overlord/http/OverlordResourceTest.java | 56 +- .../indexing/overlord/http/OverlordTest.java | 3 +- ...> OverlordSecurityResourceFilterTest.java} | 29 +- .../supervisor/SupervisorResourceTest.java | 2 +- .../query/QueryInterruptedException.java | 1 + .../io/druid/client/BrokerServerView.java | 6 +- .../http/AbstractHttpClientProvider.java | 17 + .../io/druid/guice/http/HttpClientModule.java | 23 +- .../guice/security/AuthenticatorModule.java | 53 ++ .../security/AuthorizationManagerModule.java | 53 ++ .../druid/initialization/Initialization.java | 6 + .../io/druid/server/BrokerQueryResource.java | 8 +- .../io/druid/server/ClientInfoResource.java | 41 +- .../java/io/druid/server/QueryLifecycle.java | 82 ++- .../druid/server/QueryLifecycleFactory.java | 9 +- .../java/io/druid/server/QueryResource.java | 52 +- .../server/http/DatasourcesResource.java | 11 +- .../druid/server/http/IntervalsResource.java | 15 +- .../druid/server/http/InventoryViewUtils.java | 18 +- .../druid/server/http/MetadataResource.java | 47 +- .../http/security/AbstractResourceFilter.java | 19 +- .../http/security/ConfigResourceFilter.java | 29 +- .../security/DatasourceResourceFilter.java | 63 +- .../http/security/RulesResourceFilter.java | 27 +- .../http/security/SecurityResourceFilter.java | 85 +++ .../http/security/StateResourceFilter.java | 29 +- .../AuthorizationManagerMapperModule.java | 102 +++ .../NoopAuthorizationManagerModule.java | 46 ++ .../java/io/druid/server/security/Action.java | 13 +- .../io/druid/server/security/AuthConfig.java | 86 ++- .../server/security/AuthenticationUtils.java | 105 +++ .../druid/server/security/Authenticator.java | 101 +++ .../server/security/AuthorizationInfo.java | 44 -- .../server/security/AuthorizationManager.java | 48 ++ .../security/AuthorizationManagerMapper.java | 39 ++ .../server/security/AuthorizationUtils.java | 346 ++++++++++ .../security/DefaultAuthorizationManager.java | 35 + .../server/security/NoopAuthenticator.java | 118 ++++ ...nfo.java => NoopAuthorizationManager.java} | 16 +- .../PreResponseAuthorizationCheckFilter.java | 265 ++++++++ .../io/druid/server/security/Resource.java | 11 +- .../druid/server/security/ResourceAction.java | 51 ++ .../druid/server/security/ResourceType.java | 13 +- .../security/SecuritySanityCheckFilter.java | 102 +++ .../security/UnsecuredResourceFilter.java | 56 ++ .../io/druid/client/BrokerServerViewTest.java | 4 +- .../AsyncQueryForwardingServletTest.java | 13 + .../druid/server/ClientInfoResourceTest.java | 8 +- .../io/druid/server/QueryResourceTest.java | 148 +++-- .../server/http/DatasourcesResourceTest.java | 73 ++- .../server/http/IntervalsResourceTest.java | 9 +- .../security/ResourceFilterTestHelper.java | 42 +- .../security/SecurityResourceFilterTest.java | 40 +- .../server/initialization/JettyQosTest.java | 13 + .../server/initialization/JettyTest.java | 13 + .../main/java/io/druid/cli/CliOverlord.java | 41 ++ .../src/main/java/io/druid/cli/CliPeon.java | 2 +- .../CoordinatorJettyServerInitializer.java | 51 ++ .../MiddleManagerJettyServerInitializer.java | 35 + .../cli/QueryJettyServerInitializer.java | 32 + .../cli/RouterJettyServerInitializer.java | 35 + .../sql/avatica/DruidAvaticaHandler.java | 3 - .../io/druid/sql/avatica/DruidConnection.java | 5 + .../java/io/druid/sql/avatica/DruidMeta.java | 76 ++- .../io/druid/sql/avatica/DruidStatement.java | 10 +- .../sql/calcite/planner/DruidPlanner.java | 118 +++- .../sql/calcite/planner/PlannerFactory.java | 13 +- .../sql/calcite/rel/DruidNestedGroupBy.java | 6 + .../druid/sql/calcite/rel/DruidQueryRel.java | 8 + .../io/druid/sql/calcite/rel/DruidRel.java | 7 + .../druid/sql/calcite/rel/DruidSemiJoin.java | 10 + .../io/druid/sql/calcite/rel/QueryMaker.java | 5 +- .../druid/sql/calcite/schema/DruidSchema.java | 5 +- .../java/io/druid/sql/http/SqlResource.java | 12 +- .../sql/avatica/DruidAvaticaHandlerTest.java | 22 +- .../druid/sql/avatica/DruidStatementTest.java | 13 +- .../druid/sql/calcite/CalciteQueryTest.java | 5 +- .../sql/calcite/http/SqlResourceTest.java | 7 +- .../druid/sql/calcite/util/CalciteTests.java | 4 +- 100 files changed, 4117 insertions(+), 836 deletions(-) create mode 100644 docs/content/configuration/auth.md create mode 100644 extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java create mode 100644 extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java delete mode 100644 extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterHolder.java rename indexing-service/src/test/java/io/druid/indexing/overlord/http/security/{SecurityResourceFilterTest.java => OverlordSecurityResourceFilterTest.java} (87%) create mode 100644 server/src/main/java/io/druid/guice/security/AuthenticatorModule.java create mode 100644 server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java create mode 100644 server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java create mode 100644 server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java create mode 100644 server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java create mode 100644 server/src/main/java/io/druid/server/security/AuthenticationUtils.java create mode 100644 server/src/main/java/io/druid/server/security/Authenticator.java delete mode 100644 server/src/main/java/io/druid/server/security/AuthorizationInfo.java create mode 100644 server/src/main/java/io/druid/server/security/AuthorizationManager.java create mode 100644 server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java create mode 100644 server/src/main/java/io/druid/server/security/AuthorizationUtils.java create mode 100644 server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java create mode 100644 server/src/main/java/io/druid/server/security/NoopAuthenticator.java rename server/src/main/java/io/druid/server/security/{SystemAuthorizationInfo.java => NoopAuthorizationManager.java} (69%) create mode 100644 server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java create mode 100644 server/src/main/java/io/druid/server/security/ResourceAction.java create mode 100644 server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java create mode 100644 server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java index 0f831f77eeef..e054d5a89c2f 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java @@ -38,6 +38,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.segment.QueryableIndex; +import io.druid.server.security.AuthConfig; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.planner.PlannerFactory; @@ -118,7 +119,9 @@ public void setup() throws Exception CalciteTests.createMockQueryLifecycleFactory(walker), CalciteTests.createOperatorTable(), CalciteTests.createExprMacroTable(), - plannerConfig + plannerConfig, + new AuthConfig(), + null ); groupByQuery = GroupByQuery .builder() diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md new file mode 100644 index 000000000000..f1972037a7a0 --- /dev/null +++ b/docs/content/configuration/auth.md @@ -0,0 +1,48 @@ +--- +layout: doc_page +--- + +# Authentication and Authorization + +|Property|Type|Description|Default|Required| +|--------|-----------|-------| +|`druid.auth.enabled`|boolean|Determines if authentication and authorization checks will be performed on requests.|false|no| +|`druid.auth.authenticationChainPath`|String|Path to a file containing a JSON list of Authenticator objects|null|yes| +|`druid.auth.internalAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes| +|`druid.auth.authorizationManager`|String|Type of the AuthorizationManager to be used for authorization checks.|"default"|no| + +## Enabling Authentication/Authorization + +## Authentication Chain +Authentication decisions are handled by a chain of Authenticator instances. A request will be checked by Authenticators in the sequence defined by the `druid.auth.authenticationChainPath` file. + +Authenticator implementions are provided by extensions. + +For example, the following authentication chain definition enables the Kerberos and HTTP Basic authenticators, from the `druid-kerberos` and `druid-basic-security` core extensions, respectively: + +```json +[ +{ + "type": "kerberos" +}, +{ + "type": "basic" +} +] +``` + +## Internal Authenticator +The `druid.auth.internalAuthenticator` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a broker node communicates with historical nodes for query processing). + +The Authenticator chosen for this property must also be present in the Authentication Chain. + +## Authorization Manager +Authorization decisions are handled by an AuthorizationManager. The `druid.auth.authorizationManager` property determines what type of AuthorizationManager will be used. + +There are two built-in AuthorizationManagers, "default" and "noop". Other implementations are provided by extensions. + +### Default Authorization Manager +The default AuthorizationManager with type name "default" rejects all requests. + +### No-op Authorization Manager +The no-op AuthorizationManager with type name "noop" accepts all requests. diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java index 7ed11fe59cc0..be695c92ccd0 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java @@ -27,14 +27,23 @@ public class AuthenticationKerberosConfig { @JsonProperty private final String principal; + @JsonProperty private final String keytab; + @JsonProperty + private final String namespace; + @JsonCreator - public AuthenticationKerberosConfig(@JsonProperty("principal") String principal, @JsonProperty("keytab") String keytab) + public AuthenticationKerberosConfig( + @JsonProperty("principal") String principal, + @JsonProperty("keytab") String keytab, + @JsonProperty("namespace") String namespace + ) { this.principal = principal; this.keytab = keytab; + this.namespace = namespace; } @JsonProperty @@ -49,13 +58,19 @@ public String getKeytab() return keytab; } + @JsonProperty + public String getNamespace() + { + return namespace; + } + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof AuthenticationKerberosConfig)) { + if (o == null || getClass() != o.getClass()) { return false; } @@ -64,7 +79,10 @@ public boolean equals(Object o) if (getPrincipal() != null ? !getPrincipal().equals(that.getPrincipal()) : that.getPrincipal() != null) { return false; } - return getKeytab() != null ? getKeytab().equals(that.getKeytab()) : that.getKeytab() == null; + if (getKeytab() != null ? !getKeytab().equals(that.getKeytab()) : that.getKeytab() != null) { + return false; + } + return getNamespace() != null ? getNamespace().equals(that.getNamespace()) : that.getNamespace() == null; } @@ -73,6 +91,7 @@ public int hashCode() { int result = getPrincipal() != null ? getPrincipal().hashCode() : 0; result = 31 * result + (getKeytab() != null ? getKeytab().hashCode() : 0); + result = 31 * result + (getNamespace() != null ? getNamespace().hashCode() : 0); return result; } } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java new file mode 100644 index 000000000000..2a564104ef6e --- /dev/null +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java @@ -0,0 +1,229 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.security.kerberos; + +import io.druid.java.util.common.logger.Logger; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.apache.hadoop.security.authentication.server.AuthenticationToken; +import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler; +import org.apache.hadoop.security.authentication.util.KerberosName; +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSCredential; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.Oid; + +import javax.security.auth.Subject; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.regex.Pattern; + +public class DruidKerberosAuthenticationHandler extends KerberosAuthenticationHandler +{ + private static final Logger log = new Logger(DruidKerberosAuthenticationHandler.class); + + private String keytab; + private GSSManager gssManager; + private Subject serverSubject = new Subject(); + private List loginContexts = new ArrayList(); + + @Override + public void destroy() + { + keytab = null; + serverSubject = null; + for (LoginContext loginContext : loginContexts) { + try { + loginContext.logout(); + } + catch (LoginException ex) { + log.warn(ex.getMessage(), ex); + } + } + loginContexts.clear(); + } + + @Override + public void init(Properties config) throws ServletException + { + try { + String principal = config.getProperty(PRINCIPAL); + if (principal == null || principal.trim().length() == 0) { + throw new ServletException("Principal not defined in configuration"); + } + keytab = config.getProperty(KEYTAB, keytab); + if (keytab == null || keytab.trim().length() == 0) { + throw new ServletException("Keytab not defined in configuration"); + } + if (!new File(keytab).exists()) { + throw new ServletException("Keytab does not exist: " + keytab); + } + + // use all SPNEGO principals in the keytab if a principal isn't + // specifically configured + final String[] spnegoPrincipals; + if (principal.equals("*")) { + spnegoPrincipals = KerberosUtil.getPrincipalNames( + keytab, Pattern.compile("HTTP/.*")); + if (spnegoPrincipals.length == 0) { + throw new ServletException("Principals do not exist in the keytab"); + } + } else { + spnegoPrincipals = new String[]{principal}; + } + + String nameRules = config.getProperty(NAME_RULES, null); + if (nameRules != null) { + KerberosName.setRules(nameRules); + } + + for (String spnegoPrincipal : spnegoPrincipals) { + log.info("Login using keytab {}, for principal {}", + keytab, spnegoPrincipal + ); + final KerberosAuthenticator.DruidKerberosConfiguration kerberosConfiguration = + new KerberosAuthenticator.DruidKerberosConfiguration(keytab, spnegoPrincipal); + final LoginContext loginContext = + new LoginContext("", serverSubject, null, kerberosConfiguration); + try { + loginContext.login(); + } + catch (LoginException le) { + log.warn("Failed to login as [{}]", spnegoPrincipal, le); + throw new AuthenticationException(le); + } + loginContexts.add(loginContext); + } + try { + gssManager = Subject.doAs(serverSubject, new PrivilegedExceptionAction() + { + + @Override + public GSSManager run() throws Exception + { + return GSSManager.getInstance(); + } + }); + } + catch (PrivilegedActionException ex) { + throw ex.getException(); + } + } + catch (Exception ex) { + throw new ServletException(ex); + } + } + + @Override + public AuthenticationToken authenticate(HttpServletRequest request, final HttpServletResponse response) + throws IOException, AuthenticationException + { + AuthenticationToken token = null; + String authorization = request.getHeader(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.AUTHORIZATION); + + if (authorization == null + || !authorization.startsWith(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE)) { + return null; + } else { + authorization = authorization.substring(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE + .length()).trim(); + final Base64 base64 = new Base64(0); + final byte[] clientToken = base64.decode(authorization); + final String serverName = request.getServerName(); + try { + token = Subject.doAs(serverSubject, new PrivilegedExceptionAction() + { + + @Override + public AuthenticationToken run() throws Exception + { + AuthenticationToken token = null; + GSSContext gssContext = null; + GSSCredential gssCreds = null; + try { + gssCreds = gssManager.createCredential( + gssManager.createName( + KerberosUtil.getServicePrincipal("HTTP", serverName), + KerberosUtil.getOidInstance("NT_GSS_KRB5_PRINCIPAL") + ), + GSSCredential.INDEFINITE_LIFETIME, + new Oid[]{ + KerberosUtil.getOidInstance("GSS_SPNEGO_MECH_OID"), + KerberosUtil.getOidInstance("GSS_KRB5_MECH_OID") + }, + GSSCredential.ACCEPT_ONLY + ); + gssContext = gssManager.createContext(gssCreds); + byte[] serverToken = gssContext.acceptSecContext(clientToken, 0, clientToken.length); + if (serverToken != null && serverToken.length > 0) { + String authenticate = base64.encodeToString(serverToken); + response.setHeader( + org.apache.hadoop.security.authentication.client.KerberosAuthenticator.WWW_AUTHENTICATE, + org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE + + " " + + authenticate + ); + } + if (!gssContext.isEstablished()) { + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + log.trace("SPNEGO in progress"); + } else { + String clientPrincipal = gssContext.getSrcName().toString(); + KerberosName kerberosName = new KerberosName(clientPrincipal); + String userName = kerberosName.getShortName(); + token = new AuthenticationToken(userName, clientPrincipal, getType()); + response.setStatus(HttpServletResponse.SC_OK); + log.trace("SPNEGO completed for principal [{}]", clientPrincipal); + } + } + finally { + if (gssContext != null) { + gssContext.dispose(); + } + if (gssCreds != null) { + gssCreds.dispose(); + } + } + return token; + } + }); + } + catch (PrivilegedActionException ex) { + if (ex.getException() instanceof IOException) { + throw (IOException) ex.getException(); + } else { + throw new AuthenticationException(ex.getException()); + } + } + } + return token; + } +} diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java index 30d4df8b5ef9..be6d36271f58 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java @@ -20,19 +20,17 @@ package io.druid.security.kerberos; import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import com.google.inject.multibindings.Multibinder; -import com.metamx.http.client.HttpClient; +import com.google.inject.Provides; +import com.google.inject.name.Named; import io.druid.guice.JsonConfigProvider; -import io.druid.guice.LazySingleton; -import io.druid.guice.annotations.Client; -import io.druid.guice.annotations.Global; -import io.druid.guice.http.HttpClientModule; -import io.druid.guice.http.JettyHttpClientModule; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Self; import io.druid.initialization.DruidModule; -import io.druid.server.initialization.jetty.ServletFilterHolder; -import io.druid.server.router.Router; +import io.druid.server.DruidNode; +import io.druid.server.security.Authenticator; import java.util.List; @@ -45,6 +43,9 @@ public class DruidKerberosModule implements DruidModule public List getJacksonModules() { return ImmutableList.of( + new SimpleModule("DruidKerberos").registerSubtypes( + KerberosAuthenticator.class + ) ); } @@ -53,24 +54,17 @@ public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", AuthenticationKerberosConfig.class); JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); + } - Multibinder.newSetBinder(binder, ServletFilterHolder.class) - .addBinding() - .to(SpnegoFilterHolder.class); - - binder.bind(HttpClient.class) - .annotatedWith(Global.class) - .toProvider(new KerberosHttpClientProvider(new HttpClientModule.HttpClientProvider(Global.class))) - .in(LazySingleton.class); - - binder.bind(HttpClient.class) - .annotatedWith(Client.class) - .toProvider(new KerberosHttpClientProvider(new HttpClientModule.HttpClientProvider(Client.class))) - .in(LazySingleton.class); - - binder.bind(org.eclipse.jetty.client.HttpClient.class) - .annotatedWith(Router.class) - .toProvider(new KerberosJettyHttpClientProvider(new JettyHttpClientModule.HttpClientProvider(Router.class))) - .in(LazySingleton.class); + @Provides + @ManageLifecycle + @Named("kerberos") + public Authenticator getAuthenticator( + AuthenticationKerberosConfig authenticationKerberosConfig, + SpnegoFilterConfig spnegoFilterConfig, + @Self DruidNode druidNode + ) + { + return new KerberosAuthenticator(authenticationKerberosConfig, spnegoFilterConfig, druidNode); } } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java index d171821051a1..872f820c38c5 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java @@ -73,7 +73,7 @@ public static String kerberosChallenge(String server) throws AuthenticationExcep // Create a GSSContext for authentication with the service. // We're passing client credentials as null since we want them to be read from the Subject. GSSContext gssContext = - manager.createContext(serverName.canonicalize(mechOid), mechOid, null, GSSContext.DEFAULT_LIFETIME); + manager.createContext(serverName.canonicalize(mechOid), mechOid, null, GSSContext.DEFAULT_LIFETIME); gssContext.requestMutualAuth(true); gssContext.requestCredDeleg(true); // Establish context @@ -92,7 +92,7 @@ public static String kerberosChallenge(String server) throws AuthenticationExcep } public static void authenticateIfRequired(AuthenticationKerberosConfig config) - throws IOException + throws IOException { String principal = config.getPrincipal(); String keytab = config.getKeytab(); diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java new file mode 100644 index 000000000000..d5feefe686ec --- /dev/null +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -0,0 +1,599 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.security.kerberos; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Throwables; +import com.metamx.http.client.HttpClient; +import io.druid.guice.annotations.Self; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.DruidNode; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.authentication.client.AuthenticatedURL; +import org.apache.hadoop.security.authentication.client.AuthenticationException; +import org.apache.hadoop.security.authentication.server.AuthenticationFilter; +import org.apache.hadoop.security.authentication.server.AuthenticationToken; +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.apache.hadoop.security.authentication.util.Signer; +import org.apache.hadoop.security.authentication.util.SignerException; +import org.apache.hadoop.security.authentication.util.SignerSecretProvider; +import sun.security.krb5.EncryptedData; +import sun.security.krb5.EncryptionKey; +import sun.security.krb5.internal.APReq; +import sun.security.krb5.internal.EncTicketPart; +import sun.security.krb5.internal.Krb5; +import sun.security.krb5.internal.Ticket; +import sun.security.krb5.internal.crypto.KeyUsage; +import sun.security.util.DerInputStream; +import sun.security.util.DerValue; + +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosKey; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.kerberos.KeyTab; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginContext; +import javax.servlet.DispatcherType; +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletContext; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.Cookie; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletRequestWrapper; +import javax.servlet.http.HttpServletResponse; +import java.io.File; +import java.io.IOException; +import java.security.Principal; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +@JsonTypeName("kerberos") +public class KerberosAuthenticator implements Authenticator +{ + private static final Logger log = new Logger(KerberosAuthenticator.class); + private static final Pattern HADOOP_AUTH_COOKIE_REGEX = Pattern.compile(".*p=(\\S+)&t=.*"); + + private final SpnegoFilterConfig spnegoConfig; + private final AuthenticationKerberosConfig authConfig; + private final DruidNode node; + private LoginContext loginContext; + + @JsonCreator + public KerberosAuthenticator( + @JacksonInject AuthenticationKerberosConfig authConfig, + @JacksonInject SpnegoFilterConfig spnegoConfig, + @Self DruidNode node + ) + { + this.spnegoConfig = spnegoConfig; + this.authConfig = authConfig; + this.node = node; + } + + @Override + public String getTypeName() + { + return "kerberos"; + } + + @Override + public String getNamespace() + { + return authConfig.getNamespace(); + } + + @Override + public Filter getFilter() + { + return new AuthenticationFilter() + { + private Signer mySigner; + + @Override + public void init(FilterConfig filterConfig) throws ServletException + { + ClassLoader prevLoader = Thread.currentThread().getContextClassLoader(); + try { + // AuthenticationHandler is created during Authenticationfilter.init using reflection with thread context class loader. + // In case of druid since the class is actually loaded as an extension and filter init is done in main thread. + // We need to set the classloader explicitly to extension class loader. + Thread.currentThread().setContextClassLoader(AuthenticationFilter.class.getClassLoader()); + super.init(filterConfig); + String configPrefix = filterConfig.getInitParameter(CONFIG_PREFIX); + configPrefix = (configPrefix != null) ? configPrefix + "." : ""; + Properties config = getConfiguration(configPrefix, filterConfig); + String signatureSecret = config.getProperty(configPrefix + SIGNATURE_SECRET); + if (signatureSecret == null) { + signatureSecret = Long.toString(new Random().nextLong()); + log.warn("'signature.secret' configuration not set, using a random value as secret"); + } + final byte[] secretBytes = StringUtils.toUtf8(signatureSecret); + SignerSecretProvider signerSecretProvider = new SignerSecretProvider() + { + @Override + public void init(Properties config, ServletContext servletContext, long tokenValidity) throws Exception + { + + } + + @Override + public byte[] getCurrentSecret() + { + return secretBytes; + } + + @Override + public byte[][] getAllSecrets() + { + return new byte[][]{secretBytes}; + } + }; + mySigner = new Signer(signerSecretProvider); + } + finally { + Thread.currentThread().setContextClassLoader(prevLoader); + } + } + + @Override + protected AuthenticationToken getToken(HttpServletRequest request) throws IOException, AuthenticationException + { + AuthenticationToken token = null; + String tokenStr = null; + Cookie[] cookies = request.getCookies(); + if (cookies != null) { + for (Cookie cookie : cookies) { + if (cookie.getName().equals(AuthenticatedURL.AUTH_COOKIE)) { + tokenStr = cookie.getValue(); + try { + tokenStr = mySigner.verifyAndExtract(tokenStr); + } + catch (SignerException ex) { + throw new AuthenticationException(ex); + } + break; + } + } + } + if (tokenStr != null) { + token = AuthenticationToken.parse(tokenStr); + if (!token.getType().equals(getAuthenticationHandler().getType())) { + throw new AuthenticationException("Invalid AuthenticationToken type"); + } + if (token.isExpired()) { + throw new AuthenticationException("AuthenticationToken expired"); + } + } + return token; + } + + @Override + public void doFilter( + ServletRequest request, ServletResponse response, FilterChain filterChain + ) throws IOException, ServletException + { + HttpServletRequest httpReq = (HttpServletRequest) request; + + // If there's already an auth token, then we have authenticated already, skip this. + if (request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) != null) { + filterChain.doFilter(request, response); + return; + } + + if (loginContext == null) { + initializeKerberosLogin(); + } + + String path = ((HttpServletRequest) request).getRequestURI(); + if (isExcluded(path)) { + filterChain.doFilter(request, response); + } else { + String clientPrincipal = null; + try { + Cookie[] cookies = httpReq.getCookies(); + if (cookies == null) { + clientPrincipal = getPrincipalFromRequestNew((HttpServletRequest) request); + } else { + clientPrincipal = null; + for (Cookie cookie : cookies) { + if ("hadoop.auth".equals(cookie.getName())) { + Matcher matcher = HADOOP_AUTH_COOKIE_REGEX.matcher(cookie.getValue()); + if (matcher.matches()) { + clientPrincipal = matcher.group(1); + break; + } + } + } + } + } + catch (Exception ex) { + clientPrincipal = null; + } + + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN, clientPrincipal); + request.setAttribute(AuthConfig.DRUID_AUTH_NAMESPACE, getNamespace()); + } + + doFilterSuper(request, response, filterChain); + } + + private void doFilterSuper(ServletRequest request, ServletResponse response, FilterChain filterChain) + throws IOException, ServletException + { + boolean unauthorizedResponse = true; + int errCode = HttpServletResponse.SC_UNAUTHORIZED; + AuthenticationException authenticationEx = null; + HttpServletRequest httpRequest = (HttpServletRequest) request; + HttpServletResponse httpResponse = (HttpServletResponse) response; + boolean isHttps = "https".equals(httpRequest.getScheme()); + try { + boolean newToken = false; + AuthenticationToken token; + try { + token = getToken(httpRequest); + } + catch (AuthenticationException ex) { + log.warn("AuthenticationToken ignored: " + ex.getMessage()); + // will be sent back in a 401 unless filter authenticates + authenticationEx = ex; + token = null; + } + if (getAuthenticationHandler().managementOperation(token, httpRequest, httpResponse)) { + if (token == null) { + if (log.isDebugEnabled()) { + log.debug("Request [{}] triggering authentication", getRequestURL(httpRequest)); + } + token = getAuthenticationHandler().authenticate(httpRequest, httpResponse); + if (token != null && token.getExpires() != 0 && + token != AuthenticationToken.ANONYMOUS) { + token.setExpires(System.currentTimeMillis() + getValidity() * 1000); + } + newToken = true; + } + if (token != null) { + unauthorizedResponse = false; + if (log.isDebugEnabled()) { + log.debug("Request [{}] user [{}] authenticated", getRequestURL(httpRequest), token.getUserName()); + } + final AuthenticationToken authToken = token; + httpRequest = new HttpServletRequestWrapper(httpRequest) + { + + @Override + public String getAuthType() + { + return authToken.getType(); + } + + @Override + public String getRemoteUser() + { + return authToken.getUserName(); + } + + @Override + public Principal getUserPrincipal() + { + return (authToken != AuthenticationToken.ANONYMOUS) ? authToken : null; + } + }; + if (newToken && !token.isExpired() && token != AuthenticationToken.ANONYMOUS) { + String signedToken = mySigner.sign(token.toString()); + createAuthCookie(httpResponse, signedToken, getCookieDomain(), + getCookiePath(), token.getExpires(), isHttps + ); + } + doFilter(filterChain, httpRequest, httpResponse); + } + } else { + unauthorizedResponse = false; + } + } + catch (AuthenticationException ex) { + // exception from the filter itself is fatal + errCode = HttpServletResponse.SC_FORBIDDEN; + authenticationEx = ex; + if (log.isDebugEnabled()) { + log.debug("Authentication exception: " + ex.getMessage(), ex); + } else { + log.warn("Authentication exception: " + ex.getMessage()); + } + } + if (unauthorizedResponse) { + if (!httpResponse.isCommitted()) { + createAuthCookie(httpResponse, "", getCookieDomain(), + getCookiePath(), 0, isHttps + ); + // If response code is 401. Then WWW-Authenticate Header should be + // present.. reset to 403 if not found.. + if ((errCode == HttpServletResponse.SC_UNAUTHORIZED) + && (!httpResponse.containsHeader( + org.apache.hadoop.security.authentication.client.KerberosAuthenticator.WWW_AUTHENTICATE))) { + errCode = HttpServletResponse.SC_FORBIDDEN; + } + if (authenticationEx == null) { + // Don't send an error response here, unlike the base AuthenticationFilter implementation. + // This request did not use Kerberos auth. + // Instead, we will send an error response in PreResponseAuthorizationCheckFilter to allow + // other Authenticator implementations to check the request. + filterChain.doFilter(request, response); + } else { + // Do send an error response here, we attempted Kerberos authentication and failed. + httpResponse.sendError(errCode, authenticationEx.getMessage()); + } + } + } + } + }; + } + + @Override + public Class getFilterClass() + { + return null; + } + + @Override + public Map getInitParameters() + { + Map params = new HashMap(); + try { + params.put( + "kerberos.principal", + SecurityUtil.getServerPrincipal(spnegoConfig.getPrincipal(), node.getHost()) + ); + params.put("kerberos.keytab", spnegoConfig.getKeytab()); + //params.put(AuthenticationFilter.AUTH_TYPE, "kerberos"); + params.put(AuthenticationFilter.AUTH_TYPE, DruidKerberosAuthenticationHandler.class.getName()); + params.put("kerberos.name.rules", spnegoConfig.getAuthToLocal()); + if (spnegoConfig.getCookieSignatureSecret() != null) { + params.put("signature.secret", spnegoConfig.getCookieSignatureSecret()); + } + } + catch (IOException e) { + Throwables.propagate(e); + } + return params; + } + + @Override + public String getPath() + { + return "/*"; + } + + @Override + public EnumSet getDispatcherType() + { + return null; + } + + @Override + public String getAuthChallengeHeader() + { + return "Negotiate"; + } + + @Override + public boolean authenticateJDBCContext(Map context) + { + throw new UnsupportedOperationException("JDBC Kerberos auth not supported yet"); + } + + @Override + public HttpClient createInternalClient(HttpClient baseClient) + { + return new KerberosHttpClient(baseClient, authConfig); + } + + private boolean isExcluded(String path) + { + for (String excluded : spnegoConfig.getExcludedPaths()) { + if (path.startsWith(excluded)) { + return true; + } + } + return false; + } + + + /** + * Kerberos context configuration for the JDK GSS library. + */ + public static class DruidKerberosConfiguration extends Configuration + { + private String keytab; + private String principal; + + public DruidKerberosConfiguration(String keytab, String principal) + { + this.keytab = keytab; + this.principal = principal; + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) + { + Map options = new HashMap(); + if (System.getProperty("java.vendor").contains("IBM")) { + options.put( + "useKeytab", + keytab.startsWith("file://") ? keytab : "file://" + keytab + ); + options.put("principal", principal); + options.put("credsType", "acceptor"); + } else { + options.put("keyTab", keytab); + options.put("principal", principal); + options.put("useKeyTab", "true"); + options.put("storeKey", "true"); + options.put("doNotPrompt", "true"); + options.put("useTicketCache", "true"); + options.put("renewTGT", "true"); + options.put("isInitiator", "false"); + } + options.put("refreshKrb5Config", "true"); + String ticketCache = System.getenv("KRB5CCNAME"); + if (ticketCache != null) { + if (System.getProperty("java.vendor").contains("IBM")) { + options.put("useDefaultCcache", "true"); + // The first value searched when "useDefaultCcache" is used. + System.setProperty("KRB5CCNAME", ticketCache); + options.put("renewTGT", "true"); + options.put("credsType", "both"); + } else { + options.put("ticketCache", ticketCache); + } + } + if (log.isDebugEnabled()) { + options.put("debug", "true"); + } + + return new AppConfigurationEntry[]{ + new AppConfigurationEntry( + KerberosUtil.getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + options + ), + }; + } + } + + private String getPrincipalFromRequestNew(HttpServletRequest req) + { + String authorization = req.getHeader(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.AUTHORIZATION); + if (authorization == null + || !authorization.startsWith(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE)) { + return null; + } else { + authorization = authorization.substring(org.apache.hadoop.security.authentication.client.KerberosAuthenticator.NEGOTIATE + .length()).trim(); + final Base64 base64 = new Base64(0); + final byte[] clientToken = base64.decode(authorization); + try { + DerInputStream ticketStream = new DerInputStream(clientToken); + DerValue[] values = ticketStream.getSet(clientToken.length, true); + + // see this link for AP-REQ format: https://tools.ietf.org/html/rfc1510#section-5.5.1 + for (DerValue value : values) { + if (isValueAPReq(value)) { + APReq apReq = new APReq(value); + Ticket ticket = apReq.ticket; + EncryptedData encData = ticket.encPart; + int eType = encData.getEType(); + + // find the server's key + EncryptionKey finalKey = null; + Subject serverSubj = loginContext.getSubject(); + Set serverCreds = serverSubj.getPrivateCredentials(Object.class); + for (Object cred : serverCreds) { + if (cred instanceof KeyTab) { + KeyTab serverKeyTab = (KeyTab) cred; + KerberosPrincipal serverPrincipal = new KerberosPrincipal(spnegoConfig.getPrincipal()); + KerberosKey[] serverKeys = serverKeyTab.getKeys(serverPrincipal); + for (KerberosKey key : serverKeys) { + if (key.getKeyType() == eType) { + finalKey = new EncryptionKey(key.getKeyType(), key.getEncoded()); + break; + } + } + } + } + + if (finalKey == null) { + log.error("Could not find matching key from server creds."); + return null; + } + + // decrypt the ticket with the server's key + byte[] decryptedBytes = encData.decrypt(finalKey, KeyUsage.KU_TICKET); + decryptedBytes = encData.reset(decryptedBytes); + EncTicketPart decrypted = new EncTicketPart(decryptedBytes); + String clientPrincipal = decrypted.cname.toString(); + return clientPrincipal; + } + } + } + catch (Exception ex) { + Throwables.propagate(ex); + } + } + + return null; + } + + private boolean isValueAPReq(DerValue value) + { + return value.isConstructed((byte) Krb5.KRB_AP_REQ); + } + + private void initializeKerberosLogin() throws ServletException + { + String principal; + String keytab; + + try { + principal = SecurityUtil.getServerPrincipal(spnegoConfig.getPrincipal(), node.getHost()); + if (principal == null || principal.trim().length() == 0) { + throw new ServletException("Principal not defined in configuration"); + } + keytab = spnegoConfig.getKeytab(); + if (keytab == null || keytab.trim().length() == 0) { + throw new ServletException("Keytab not defined in configuration"); + } + if (!new File(keytab).exists()) { + throw new ServletException("Keytab does not exist: " + keytab); + } + + Set principals = new HashSet(); + principals.add(new KerberosPrincipal(principal)); + Subject subject = new Subject(false, principals, new HashSet(), new HashSet()); + + DruidKerberosConfiguration kerberosConfiguration = new DruidKerberosConfiguration(keytab, principal); + + log.info("Login using keytab " + keytab + ", for principal " + principal); + loginContext = new LoginContext("", subject, null, kerberosConfiguration); + loginContext.login(); + + log.info("Initialized, principal %s from keytab %s", principal, keytab); + } + catch (Exception ex) { + throw new ServletException(ex); + } + } +} diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java index 9ce8f6ab098f..4dd4f9b01f97 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterConfig.java @@ -47,11 +47,11 @@ public class SpnegoFilterConfig @JsonCreator public SpnegoFilterConfig( - @JsonProperty("principal") String principal, - @JsonProperty("keytab") String keytab, - @JsonProperty("authToLocal") String authToLocal, - @JsonProperty("excludedPaths") List excludedPaths, - @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret + @JsonProperty("principal") String principal, + @JsonProperty("keytab") String keytab, + @JsonProperty("authToLocal") String authToLocal, + @JsonProperty("excludedPaths") List excludedPaths, + @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret ) { this.principal = principal; diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterHolder.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterHolder.java deleted file mode 100644 index ab2a3b6d4f4b..000000000000 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/SpnegoFilterHolder.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.security.kerberos; - -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import io.druid.guice.annotations.Self; -import io.druid.server.DruidNode; -import io.druid.server.initialization.jetty.ServletFilterHolder; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hadoop.security.authentication.server.AuthenticationFilter; - -import javax.servlet.DispatcherType; -import javax.servlet.Filter; -import javax.servlet.FilterChain; -import javax.servlet.FilterConfig; -import javax.servlet.ServletException; -import javax.servlet.ServletRequest; -import javax.servlet.ServletResponse; -import javax.servlet.http.HttpServletRequest; -import java.io.IOException; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; - -public class SpnegoFilterHolder implements ServletFilterHolder -{ - private final SpnegoFilterConfig config; - private final DruidNode node; - - @Inject - public SpnegoFilterHolder(SpnegoFilterConfig config, @Self DruidNode node) - { - this.config = config; - this.node = node; - } - - @Override - public Filter getFilter() - { - return new AuthenticationFilter() - { - @Override - public void init(FilterConfig filterConfig) throws ServletException - { - ClassLoader prevLoader = Thread.currentThread().getContextClassLoader(); - try { - // AuthenticationHandler is created during Authenticationfilter.init using reflection with thread context class loader. - // In case of druid since the class is actually loaded as an extension and filter init is done in main thread. - // We need to set the classloader explicitly to extension class loader. - Thread.currentThread().setContextClassLoader(AuthenticationFilter.class.getClassLoader()); - super.init(filterConfig); - } - finally { - Thread.currentThread().setContextClassLoader(prevLoader); - } - } - - @Override - public void doFilter( - ServletRequest request, ServletResponse response, FilterChain filterChain - ) throws IOException, ServletException - { - String path = ((HttpServletRequest) request).getRequestURI(); - if (isExcluded(path)) { - filterChain.doFilter(request, response); - } - super.doFilter(request, response, filterChain); - } - }; - } - - private boolean isExcluded(String path) - { - for (String excluded : config.getExcludedPaths()) { - if (path.startsWith(excluded)) { - return true; - } - } - return false; - } - - @Override - public Class getFilterClass() - { - return null; - } - - @Override - public Map getInitParameters() - { - Map params = new HashMap(); - try { - params.put( - "kerberos.principal", - SecurityUtil.getServerPrincipal(config.getPrincipal(), node.getHost()) - ); - params.put("kerberos.keytab", config.getKeytab()); - params.put(AuthenticationFilter.AUTH_TYPE, "kerberos"); - params.put("kerberos.name.rules", config.getAuthToLocal()); - if (config.getCookieSignatureSecret() != null) { - params.put("signature.secret", config.getCookieSignatureSecret()); - } - } - catch (IOException e) { - Throwables.propagate(e); - } - return params; - } - - @Override - public String getPath() - { - return "/*"; - } - - @Override - public EnumSet getDispatcherType() - { - return null; - } -} diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index 807c7b434f16..86ceea652ce2 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -44,6 +44,7 @@ import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.virtual.ExpressionVirtualColumn; +import io.druid.server.security.AuthConfig; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -132,7 +133,9 @@ public void setUp() throws Exception CalciteTests.createMockQueryLifecycleFactory(walker), operatorTable, CalciteTests.createExprMacroTable(), - plannerConfig + plannerConfig, + new AuthConfig(), + null ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index c9bcc5954c2e..109537601337 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -73,6 +73,14 @@ import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import io.druid.segment.realtime.firehose.ChatHandler; import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.server.security.Access; +import io.druid.server.security.Action; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; +import io.druid.server.security.ResourceType; import io.druid.timeline.DataSegment; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -82,6 +90,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.joda.time.DateTime; +import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; @@ -89,6 +98,7 @@ import javax.ws.rs.Path; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import java.io.IOException; @@ -129,6 +139,8 @@ public enum Status private final InputRowParser parser; private final KafkaTuningConfig tuningConfig; private final KafkaIOConfig ioConfig; + private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; private final Optional chatHandlerProvider; private final Map endOffsets = new ConcurrentHashMap<>(); @@ -193,7 +205,9 @@ public KafkaIndexTask( @JsonProperty("tuningConfig") KafkaTuningConfig tuningConfig, @JsonProperty("ioConfig") KafkaIOConfig ioConfig, @JsonProperty("context") Map context, - @JacksonInject ChatHandlerProvider chatHandlerProvider + @JacksonInject ChatHandlerProvider chatHandlerProvider, + @JacksonInject AuthConfig authConfig, + @JacksonInject AuthorizationManagerMapper authorizationManagerMapper ) { super( @@ -209,6 +223,8 @@ public KafkaIndexTask( this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); + this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); } @@ -606,8 +622,21 @@ public boolean canRestore() return true; } - @POST - @Path("/stop") + /** + * Authorizes action to be performed on this task's datasource + * + * @return authorization result + */ + private Access authorizationCheck(final HttpServletRequest req, Action action) + { + ResourceAction resourceAction = new ResourceAction( + new Resource(dataSchema.getDataSource(), ResourceType.DATASOURCE), + action + ); + + return AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizationManagerMapper); + } + @Override public void stopGracefully() { @@ -673,9 +702,37 @@ public Sequence run(final QueryPlus queryPlus, final Map r }; } + @POST + @Path("/stop") + public Response stop(@Context final HttpServletRequest req) + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); + } + } + stopGracefully(); + return Response.status(Response.Status.OK).build(); + } + @GET @Path("/status") @Produces(MediaType.APPLICATION_JSON) + public Status getStatusHTTP(@Context final HttpServletRequest req) + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.READ); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; + } + } + return status; + } + public Status getStatus() { return status; @@ -684,6 +741,18 @@ public Status getStatus() @GET @Path("/offsets/current") @Produces(MediaType.APPLICATION_JSON) + public Map getCurrentOffsets(@Context final HttpServletRequest req) + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.READ); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; + } + } + return getCurrentOffsets(); + } + public Map getCurrentOffsets() { return nextOffsets; @@ -692,6 +761,18 @@ public Map getCurrentOffsets() @GET @Path("/offsets/end") @Produces(MediaType.APPLICATION_JSON) + public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.READ); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; + } + } + return getEndOffsets(); + } + public Map getEndOffsets() { return endOffsets; @@ -701,9 +782,27 @@ public Map getEndOffsets() @Path("/offsets/end") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) + public Response setEndOffsetsHTTP( + Map offsets, + @QueryParam("resume") @DefaultValue("false") final boolean resume, + @Context final HttpServletRequest req + ) throws InterruptedException + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); + } + } + + return setEndOffsets(offsets, resume); + } + public Response setEndOffsets( Map offsets, - @QueryParam("resume") @DefaultValue("false") final boolean resume + final boolean resume ) throws InterruptedException { if (offsets == null) { @@ -769,8 +868,23 @@ public Response setEndOffsets( @POST @Path("/pause") @Produces(MediaType.APPLICATION_JSON) - public Response pause(@QueryParam("timeout") @DefaultValue("0") final long timeout) - throws InterruptedException + public Response pauseHTTP( + @QueryParam("timeout") @DefaultValue("0") final long timeout, + @Context final HttpServletRequest req + ) throws InterruptedException + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); + } + } + return pause(timeout); + } + + public Response pause(final long timeout) throws InterruptedException { if (!(status == Status.PAUSED || status == Status.READING)) { return Response.status(Response.Status.BAD_REQUEST) @@ -819,6 +933,20 @@ public Response pause(@QueryParam("timeout") @DefaultValue("0") final long timeo @POST @Path("/resume") + public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException + { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); + } + } + resume(); + return Response.status(Response.Status.OK).build(); + } + public void resume() throws InterruptedException { pauseLock.lockInterruptibly(); @@ -842,8 +970,15 @@ public void resume() throws InterruptedException @GET @Path("/time/start") @Produces(MediaType.APPLICATION_JSON) - public DateTime getStartTime() + public DateTime getStartTime(@Context final HttpServletRequest req) { + if (authConfig.isEnabled()) { + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; + } + } return startTime; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 155ce5ceff83..8e780ae7d3ae 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -70,6 +70,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.metadata.EntryExistsException; import io.druid.server.metrics.DruidMonitorSchedulerConfig; +import io.druid.server.security.AuthConfig; import org.apache.commons.codec.digest.DigestUtils; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; @@ -191,6 +192,7 @@ private static class TaskData private final KafkaSupervisorSpec spec; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final AuthConfig authConfig; private final String dataSource; private final KafkaSupervisorIOConfig ioConfig; private final KafkaSupervisorTuningConfig tuningConfig; @@ -224,7 +226,8 @@ public KafkaSupervisor( final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, final KafkaIndexTaskClientFactory taskClientFactory, final ObjectMapper mapper, - final KafkaSupervisorSpec spec + final KafkaSupervisorSpec spec, + final AuthConfig authConfig ) { this.taskStorage = taskStorage; @@ -234,6 +237,7 @@ public KafkaSupervisor( this.spec = spec; this.emitter = spec.getEmitter(); this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); + this.authConfig = spec.getAuthConfig(); this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -1419,6 +1423,8 @@ private void createKafkaTasksForGroup(int groupId, int replicas) taskTuningConfig, kafkaIOConfig, spec.getContext(), + null, + new AuthConfig(), null ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index c476b05e1053..909cfa59041e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -35,6 +35,7 @@ import io.druid.indexing.overlord.supervisor.SupervisorSpec; import io.druid.segment.indexing.DataSchema; import io.druid.server.metrics.DruidMonitorSchedulerConfig; +import io.druid.server.security.AuthConfig; import java.util.List; import java.util.Map; @@ -53,6 +54,7 @@ public class KafkaSupervisorSpec implements SupervisorSpec private final ObjectMapper mapper; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private final AuthConfig authConfig; @JsonCreator public KafkaSupervisorSpec( @@ -66,7 +68,8 @@ public KafkaSupervisorSpec( @JacksonInject KafkaIndexTaskClientFactory kafkaIndexTaskClientFactory, @JacksonInject @Json ObjectMapper mapper, @JacksonInject ServiceEmitter emitter, - @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig + @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, + @JacksonInject AuthConfig authConfig ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); @@ -100,6 +103,7 @@ public KafkaSupervisorSpec( this.mapper = mapper; this.emitter = emitter; this.monitorSchedulerConfig = monitorSchedulerConfig; + this.authConfig = authConfig; } @JsonProperty @@ -142,6 +146,11 @@ public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() return monitorSchedulerConfig; } + public AuthConfig getAuthConfig() + { + return authConfig; + } + @Override public Supervisor createSupervisor() { @@ -151,7 +160,8 @@ public Supervisor createSupervisor() indexerMetadataStorageCoordinator, kafkaIndexTaskClientFactory, mapper, - this + this, + authConfig ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 16a6963d8cfe..b7393a87dacf 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -117,6 +117,7 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.server.coordination.DataSegmentServerAnnouncer; +import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; @@ -1400,6 +1401,8 @@ private KafkaIndexTask createTask( tuningConfig, ioConfig, null, + null, + new AuthConfig(), null ); task.setPollRetryMs(POLL_RETRY_MS); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 84ef4e92c5ef..39470aa60b72 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -68,6 +68,7 @@ import io.druid.segment.realtime.FireDepartment; import io.druid.server.metrics.DruidMonitorSchedulerConfig; import io.druid.server.metrics.NoopServiceEmitter; +import io.druid.server.security.AuthConfig; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -1784,7 +1785,8 @@ public KafkaIndexTaskClient build( taskClientFactory, objectMapper, new NoopServiceEmitter(), - new DruidMonitorSchedulerConfig() + new DruidMonitorSchedulerConfig(), + new AuthConfig() ) ); } @@ -1848,6 +1850,8 @@ private KafkaIndexTask createKafkaIndexTask( false ), ImmutableMap.of(), + null, + new AuthConfig(), null ); } @@ -1881,7 +1885,7 @@ public TestableKafkaSupervisor( KafkaSupervisorSpec spec ) { - super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, mapper, spec); + super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, mapper, spec, new AuthConfig()); } @Override diff --git a/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index cf8371810e4f..ee274af1dd2f 100644 --- a/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -1 +1 @@ -io.druid.metadata.storage.mysql.MySQLMetadataStorageModule +io.druid.metadata.storage.mysql.MySQLMetadataStorageModule \ No newline at end of file diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 4b25c540b63e..cf97bbac86f7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -22,10 +22,6 @@ import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Function; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.Collections2; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -52,7 +48,6 @@ import io.druid.indexing.overlord.autoscaling.ScalingStats; import io.druid.indexing.overlord.http.security.TaskResourceFilter; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; -import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.EntryExistsException; @@ -61,8 +56,10 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import io.druid.tasklogs.TaskLogStreamer; import io.druid.timeline.DataSegment; @@ -85,7 +82,6 @@ import javax.ws.rs.core.Response; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -104,6 +100,7 @@ public class OverlordResource private final JacksonConfigManager configManager; private final AuditManager auditManager; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; private AtomicReference workerConfigRef = null; @@ -114,7 +111,8 @@ public OverlordResource( TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager, AuditManager auditManager, - AuthConfig authConfig + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper ) throws Exception { this.taskMaster = taskMaster; @@ -123,6 +121,7 @@ public OverlordResource( this.configManager = configManager; this.auditManager = auditManager; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } @POST @@ -135,19 +134,23 @@ public Response taskPost( ) { if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 final String dataSource = task.getDataSource(); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); - Access authResult = authorizationInfo.isAuthorized( + final ResourceAction resourceAction = new ResourceAction( new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE ); + + Access authResult = AuthorizationUtils.authorizeResourceAction( + req, + resourceAction, + authorizationManagerMapper + ); + if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); + return Response.status(Response.Status.FORBIDDEN) + .header("Access-Check-Result", authResult) + .entity(ImmutableMap.of("error", "Not authorized.")) + .build(); } } @@ -370,32 +373,25 @@ public Collection apply(TaskRunner taskRunner) final List allActiveTasks = taskStorageQueryAdapter.getActiveTasks(); final List activeTasks; if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final Map, Access> resourceAccessMap = new HashMap<>(); - final AuthorizationInfo authorizationInfo = - (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - activeTasks = ImmutableList.copyOf( - Iterables.filter( - allActiveTasks, - new Predicate() - { - @Override - public boolean apply(Task input) - { - Resource resource = new Resource(input.getDataSource(), ResourceType.DATASOURCE); - Action action = Action.READ; - Pair key = new Pair<>(resource, action); - if (resourceAccessMap.containsKey(key)) { - return resourceAccessMap.get(key).isAllowed(); - } else { - Access access = authorizationInfo.isAuthorized(key.lhs, key.rhs); - resourceAccessMap.put(key, access); - return access.isAllowed(); - } - } - } - ) + Function raGenerator = new Function() + { + @Override + public ResourceAction apply(Task input) + { + return new ResourceAction( + new Resource(input.getDataSource(), ResourceType.DATASOURCE), + Action.READ + ); + } + }; + + activeTasks = AuthorizationUtils.filterAuthorizedResources( + req, + allActiveTasks, + raGenerator, + authorizationManagerMapper ); + } else { activeTasks = allActiveTasks; } @@ -451,7 +447,6 @@ public Response getPendingTasks(@Context final HttpServletRequest req) public Collection apply(TaskRunner taskRunner) { if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 return securedTaskRunnerWorkItem(taskRunner.getPendingTasks(), req); } else { return taskRunner.getPendingTasks(); @@ -474,7 +469,6 @@ public Response getRunningTasks(@Context final HttpServletRequest req) public Collection apply(TaskRunner taskRunner) { if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 return securedTaskRunnerWorkItem(taskRunner.getRunningTasks(), req); } else { return taskRunner.getRunningTasks(); @@ -491,39 +485,33 @@ public Response getCompleteTasks(@Context final HttpServletRequest req) { final List recentlyFinishedTasks; if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final Map, Access> resourceAccessMap = new HashMap<>(); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - recentlyFinishedTasks = ImmutableList.copyOf( - Iterables.filter( - taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(), - new Predicate() - { - @Override - public boolean apply(TaskStatus input) - { - final String taskId = input.getId(); - final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); - if (!optionalTask.isPresent()) { - throw new WebApplicationException( - Response.serverError().entity( - StringUtils.format("No task information found for task with id: [%s]", taskId) - ).build() - ); - } - Resource resource = new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE); - Action action = Action.READ; - Pair key = new Pair<>(resource, action); - if (resourceAccessMap.containsKey(key)) { - return resourceAccessMap.get(key).isAllowed(); - } else { - Access access = authorizationInfo.isAuthorized(key.lhs, key.rhs); - resourceAccessMap.put(key, access); - return access.isAllowed(); - } - } - } - ) + Function raGenerator = new Function() + { + @Override + public ResourceAction apply(TaskStatus input) + { + final String taskId = input.getId(); + final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); + if (!optionalTask.isPresent()) { + throw new WebApplicationException( + Response.serverError().entity( + String.format("No task information found for task with id: [%s]", taskId) + ).build() + ); + } + + return new ResourceAction( + new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE), + Action.READ + ); + } + }; + + recentlyFinishedTasks = AuthorizationUtils.filterAuthorizedResources( + req, + taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(), + raGenerator, + authorizationManagerMapper ); } else { recentlyFinishedTasks = taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(); @@ -683,37 +671,33 @@ private Collection securedTaskRunnerWorkItem( HttpServletRequest req ) { - final Map, Access> resourceAccessMap = new HashMap<>(); - final AuthorizationInfo authorizationInfo = - (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - return Collections2.filter( - collectionToFilter, - new Predicate() - { - @Override - public boolean apply(TaskRunnerWorkItem input) - { - final String taskId = input.getTaskId(); - final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); - if (!optionalTask.isPresent()) { - throw new WebApplicationException( - Response.serverError().entity( - StringUtils.format("No task information found for task with id: [%s]", taskId) - ).build() - ); - } - Resource resource = new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE); - Action action = Action.READ; - Pair key = new Pair<>(resource, action); - if (resourceAccessMap.containsKey(key)) { - return resourceAccessMap.get(key).isAllowed(); - } else { - Access access = authorizationInfo.isAuthorized(key.lhs, key.rhs); - resourceAccessMap.put(key, access); - return access.isAllowed(); - } - } + Function raGenerator = new Function() + { + @Override + public ResourceAction apply(TaskRunnerWorkItem input) + { + final String taskId = input.getTaskId(); + final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); + if (!optionalTask.isPresent()) { + throw new WebApplicationException( + Response.serverError().entity( + String.format("No task information found for task with id: [%s]", taskId) + ).build() + ); } + + return new ResourceAction( + new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE), + Action.READ + ); + } + }; + + return AuthorizationUtils.filterAuthorizedResources( + req, + collectionToFilter, + raGenerator, + authorizationManagerMapper ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java index 6bab756ef33f..f1952b4bcf3f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java @@ -19,6 +19,7 @@ package io.druid.indexing.overlord.http.security; +import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -31,10 +32,11 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.security.Access; +import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ResourceAction; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.PathSegment; @@ -46,9 +48,13 @@ public class SupervisorResourceFilter extends AbstractResourceFilter private final SupervisorManager supervisorManager; @Inject - public SupervisorResourceFilter(AuthConfig authConfig, SupervisorManager supervisorManager) + public SupervisorResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper, + SupervisorManager supervisorManager + ) { - super(authConfig); + super(authConfig, authorizationManagerMapper); this.supervisorManager = supervisorManager; } @@ -56,7 +62,6 @@ public SupervisorResourceFilter(AuthConfig authConfig, SupervisorManager supervi public ContainerRequest filter(ContainerRequest request) { if (getAuthConfig().isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 final String supervisorId = Preconditions.checkNotNull( request.getPathSegments() .get( @@ -83,11 +88,6 @@ public boolean apply(PathSegment input) ); } - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); final SupervisorSpec spec = supervisorSpecOptional.get(); Preconditions.checkArgument( @@ -95,18 +95,23 @@ public boolean apply(PathSegment input) "No dataSources found to perform authorization checks" ); - for (String dataSource : spec.getDataSources()) { - Access authResult = authorizationInfo.isAuthorized( - new Resource(dataSource, ResourceType.DATASOURCE), - getAction(request) - ); - if (!authResult.isAllowed()) { - throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) - .entity( - StringUtils.format("Access-Check-Result: %s", authResult.toString()) - ) - .build()); - } + Function resourceActionFunction = getAction(request) == Action.READ ? + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR : + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR; + + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + getReq(), + spec.getDataSources(), + resourceActionFunction, + getAuthorizationManagerMapper() + ); + + if (!authResult.isAllowed()) { + throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) + .entity( + String.format("Access-Check-Result: %s", authResult.toString()) + ) + .build()); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java index 310de74a782b..c7264b259b54 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java @@ -32,8 +32,10 @@ import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import javax.ws.rs.WebApplicationException; @@ -52,9 +54,13 @@ public class TaskResourceFilter extends AbstractResourceFilter private final TaskStorageQueryAdapter taskStorageQueryAdapter; @Inject - public TaskResourceFilter(TaskStorageQueryAdapter taskStorageQueryAdapter, AuthConfig authConfig) + public TaskResourceFilter( + TaskStorageQueryAdapter taskStorageQueryAdapter, + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { - super(authConfig); + super(authConfig, authorizationManagerMapper); this.taskStorageQueryAdapter = taskStorageQueryAdapter; } @@ -62,7 +68,6 @@ public TaskResourceFilter(TaskStorageQueryAdapter taskStorageQueryAdapter, AuthC public ContainerRequest filter(ContainerRequest request) { if (getAuthConfig().isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 final String taskId = Preconditions.checkNotNull( request.getPathSegments() .get( @@ -90,15 +95,17 @@ public boolean apply(PathSegment input) } final String dataSourceName = Preconditions.checkNotNull(taskOptional.get().getDataSource()); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); - final Access authResult = authorizationInfo.isAuthorized( + final ResourceAction resourceAction = new ResourceAction( new Resource(dataSourceName, ResourceType.DATASOURCE), getAction(request) ); + + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizationManagerMapper() + ); + if (!authResult.isAllowed()) { throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) .entity( diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java index 365003573a6c..2e5baa673832 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -24,7 +24,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.inject.Inject; @@ -33,11 +32,9 @@ import io.druid.indexing.overlord.http.security.SupervisorResourceFilter; import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; -import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -62,12 +59,18 @@ public class SupervisorResource { private final TaskMaster taskMaster; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; @Inject - public SupervisorResource(TaskMaster taskMaster, AuthConfig authConfig) + public SupervisorResource( + TaskMaster taskMaster, + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { this.taskMaster = taskMaster; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } @POST @@ -82,13 +85,18 @@ public Response specPost(final SupervisorSpec spec, @Context final HttpServletRe public Response apply(SupervisorManager manager) { if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" + Preconditions.checkArgument( + spec.getDataSources() != null && spec.getDataSources().size() > 0, + "No dataSources found to perform authorization checks" ); - Access authResult = checkSupervisorAccess(authorizationInfo, spec); + + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + spec.getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizationManagerMapper + ); + if (!authResult.isAllowed()) { return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); } @@ -100,25 +108,6 @@ public Response apply(SupervisorManager manager) ); } - private Access checkSupervisorAccess(final AuthorizationInfo authorizationInfo, final SupervisorSpec spec) - { - Preconditions.checkArgument( - spec.getDataSources() != null && spec.getDataSources().size() > 0, - "No dataSources found to perform authorization checks" - ); - Access result = new Access(true); - for (String dataSource : spec.getDataSources()) { - result = authorizationInfo.isAuthorized( - new Resource(dataSource, ResourceType.DATASOURCE), - Action.WRITE - ); - if (!result.isAllowed()) { - return result; - } - } - return result; - } - @GET @Produces(MediaType.APPLICATION_JSON) public Response specGetAll(@Context final HttpServletRequest req) @@ -131,28 +120,22 @@ public Response apply(final SupervisorManager manager) { final Set supervisorIds; if (authConfig.isEnabled()) { - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); - supervisorIds = Sets.newHashSet( - Iterables.filter( - manager.getSupervisorIds(), - new Predicate() - { - @Override - public boolean apply(String id) - { - return manager.getSupervisorSpec(id).isPresent() && - checkSupervisorAccess( - authorizationInfo, - manager.getSupervisorSpec(id).get() - ).isAllowed(); - } - } - ) - ); + supervisorIds = Sets.newHashSet(); + for (String supervisorId : manager.getSupervisorIds()) { + Optional supervisorSpecOptional = manager.getSupervisorSpec(supervisorId); + if (supervisorSpecOptional.isPresent()) { + Access accessResult = AuthorizationUtils.authorizeAllResourceActions( + req, + supervisorSpecOptional.get().getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizationManagerMapper + ); + + if (accessResult.isAllowed()) { + supervisorIds.add(supervisorId); + } + } + } } else { supervisorIds = manager.getSupervisorIds(); } @@ -249,11 +232,6 @@ public Response apply(final SupervisorManager manager) { final Map> supervisorHistory; if (authConfig.isEnabled()) { - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); supervisorHistory = Maps.filterKeys( manager.getSupervisorHistory(), new Predicate() @@ -261,11 +239,17 @@ public Response apply(final SupervisorManager manager) @Override public boolean apply(String id) { - return manager.getSupervisorSpec(id).isPresent() && - checkSupervisorAccess( - authorizationInfo, - manager.getSupervisorSpec(id).get() - ).isAllowed(); + Optional supervisorSpecOptional = manager.getSupervisorSpec(id); + if (!supervisorSpecOptional.isPresent()) { + return false; + } + Access accessResult = AuthorizationUtils.authorizeAllResourceActions( + req, + supervisorSpecOptional.get().getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizationManagerMapper + ); + return accessResult.isAllowed(); } } ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index 506799085c31..0bfb897b3ae9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -33,6 +33,8 @@ import io.druid.segment.column.ColumnConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; import java.util.List; import java.util.concurrent.TimeUnit; @@ -72,6 +74,8 @@ public int columnCacheSizeBytes() .addValue(IndexIO.class, indexIO) .addValue(ObjectMapper.class, jsonMapper) .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider()) + .addValue(AuthConfig.class, new AuthConfig()) + .addValue(AuthorizationManagerMapper.class, null) ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 08b341668dd0..3400d4c8786d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -39,7 +39,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.server.security.Resource; import org.easymock.EasyMock; import org.junit.After; @@ -72,34 +73,53 @@ public void setUp() throws Exception Optional.of(taskRunner) ).anyTimes(); + AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new AuthorizationManager() + { + @Override + public Access authorize(String identity, Resource resource, Action action) + { + if (resource.getName().equals("allow")) { + return new Access(true); + } else { + return new Access(false); + } + } + + @Override + public String getNamespace() + { + return null; + } + }; + } + }; + overlordResource = new OverlordResource( taskMaster, tsqa, null, null, null, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper ); } public void expectAuthorizationTokenCheck() { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn( - new AuthorizationInfo() - { - @Override - public Access isAuthorized( - Resource resource, Action action - ) - { - if (resource.getName().equals("allow")) { - return new Access(true); - } else { - return new Access(false); - } - } - } - ); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid"); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + + req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + EasyMock.expectLastCall().anyTimes(); + + req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index 2f2a79b09908..cc746ed7e56f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -211,7 +211,8 @@ public void testOverlordRun() throws Exception null, null, null, - new AuthConfig() + new AuthConfig(), + null ); Response response = overlordResource.getLeader(); Assert.assertEquals(druidNode.getHostAndPort(), response.getEntity()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/SecurityResourceFilterTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java similarity index 87% rename from indexing-service/src/test/java/io/druid/indexing/overlord/http/security/SecurityResourceFilterTest.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java index 3eb38cf1f9ab..cb60e4ec52fb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/SecurityResourceFilterTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java @@ -35,6 +35,7 @@ import io.druid.indexing.worker.http.WorkerResource; import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.http.security.ResourceFilterTestHelper; +import io.druid.server.security.AuthorizationManagerMapper; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -49,7 +50,7 @@ import java.util.List; @RunWith(Parameterized.class) -public class SecurityResourceFilterTest extends ResourceFilterTestHelper +public class OverlordSecurityResourceFilterTest extends ResourceFilterTestHelper { @Parameterized.Parameters(name = "{index}: requestPath={0}, requestMethod={1}, resourceFilter={2}") @@ -57,9 +58,19 @@ public static Collection data() { return ImmutableList.copyOf( Iterables.concat( - getRequestPaths(OverlordResource.class, ImmutableList.>of(TaskStorageQueryAdapter.class)), - getRequestPaths(WorkerResource.class), - getRequestPaths(SupervisorResource.class, ImmutableList.>of(SupervisorManager.class)) + getRequestPaths(OverlordResource.class, ImmutableList.>of( + TaskStorageQueryAdapter.class, + AuthorizationManagerMapper.class + ) + ), + getRequestPaths(WorkerResource.class, ImmutableList.>of( + AuthorizationManagerMapper.class + )), + getRequestPaths(SupervisorResource.class, ImmutableList.>of( + SupervisorManager.class, + AuthorizationManagerMapper.class + ) + ) ) ); } @@ -75,7 +86,7 @@ public static Collection data() private TaskStorageQueryAdapter tsqa; private SupervisorManager supervisorManager; - public SecurityResourceFilterTest( + public OverlordSecurityResourceFilterTest( String requestPath, String requestMethod, ResourceFilter resourceFilter, @@ -139,7 +150,7 @@ public void testResourcesFilteringAccess() // As request object is a strict mock the ordering of expected calls matters // therefore adding the expectation below again as getEntity is called before getMethod EasyMock.expect(request.getMethod()).andReturn(requestMethod).anyTimes(); - EasyMock.replay(req, request, authorizationInfo); + EasyMock.replay(req, request, authorizationManagerMapper); resourceFilter.getRequestFilter().filter(request); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); } @@ -149,7 +160,7 @@ public void testDatasourcesResourcesFilteringNoAccess() { setUpMockExpectations(requestPath, false, requestMethod); EasyMock.expect(request.getEntity(Task.class)).andReturn(noopTask).anyTimes(); - EasyMock.replay(req, request, authorizationInfo); + EasyMock.replay(req, request, authorizationManagerMapper); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); try { resourceFilter.getRequestFilter().filter(request); @@ -165,14 +176,14 @@ public void testDatasourcesResourcesFilteringBadPath() { final String badRequestPath = requestPath.replaceAll("\\w+", "droid"); EasyMock.expect(request.getPath()).andReturn(badRequestPath).anyTimes(); - EasyMock.replay(req, request, authorizationInfo); + EasyMock.replay(req, request, authorizationManagerMapper); Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath)); } @After public void tearDown() { - EasyMock.verify(req, request, authorizationInfo); + EasyMock.verify(req, request, authorizationManagerMapper); if (tsqa != null) { EasyMock.verify(tsqa); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index c593f6986c05..6bdaec6599b0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -61,7 +61,7 @@ public class SupervisorResourceTest extends EasyMockSupport @Before public void setUp() throws Exception { - supervisorResource = new SupervisorResource(taskMaster, new AuthConfig()); + supervisorResource = new SupervisorResource(taskMaster, new AuthConfig(), null); } @Test diff --git a/processing/src/main/java/io/druid/query/QueryInterruptedException.java b/processing/src/main/java/io/druid/query/QueryInterruptedException.java index 4ca01358787e..d19b542082a3 100644 --- a/processing/src/main/java/io/druid/query/QueryInterruptedException.java +++ b/processing/src/main/java/io/druid/query/QueryInterruptedException.java @@ -45,6 +45,7 @@ public class QueryInterruptedException extends RuntimeException public static final String QUERY_TIMEOUT = "Query timeout"; public static final String QUERY_CANCELLED = "Query cancelled"; public static final String RESOURCE_LIMIT_EXCEEDED = "Resource limit exceeded"; + public static final String UNAUTHORIZED = "Unauthorized request."; public static final String UNKNOWN_EXCEPTION = "Unknown exception"; private final String errorCode; diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index 13495c552654..ea0b68ed2004 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -40,6 +40,7 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -74,6 +75,7 @@ public class BrokerServerView implements TimelineServerView private final TierSelectorStrategy tierSelectorStrategy; private final ServiceEmitter emitter; private final Predicate> segmentFilter; + private final AuthConfig authConfig; private volatile boolean initialized = false; @@ -86,7 +88,8 @@ public BrokerServerView( FilteredServerInventoryView baseView, TierSelectorStrategy tierSelectorStrategy, ServiceEmitter emitter, - final BrokerSegmentWatcherConfig segmentWatcherConfig + final BrokerSegmentWatcherConfig segmentWatcherConfig, + AuthConfig authConfig ) { this.warehouse = warehouse; @@ -99,6 +102,7 @@ public BrokerServerView( this.clients = Maps.newConcurrentMap(); this.selectors = Maps.newHashMap(); this.timelines = Maps.newHashMap(); + this.authConfig = authConfig; this.segmentFilter = new Predicate>() { diff --git a/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java b/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java index d1b771db403d..1f489f4c2f5d 100644 --- a/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java +++ b/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java @@ -27,9 +27,13 @@ import com.google.inject.Provider; import com.google.inject.TypeLiteral; import io.druid.java.util.common.lifecycle.Lifecycle; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import javax.net.ssl.SSLContext; import java.lang.annotation.Annotation; +import java.util.List; /** */ @@ -91,6 +95,19 @@ public Provider> getConfigProvider() return injector.getProvider(configKey); } + public AuthConfig getAuthConfig() + { + return injector.getInstance(AuthConfig.class); + } + + public List getAuthenticatorChain() + { + return AuthenticationUtils.getAuthenticatorChainFromConfig( + getAuthConfig().getAuthenticatorChain(), + injector + ); + } + public Provider getLifecycleProvider() { return injector.getProvider(Lifecycle.class); diff --git a/server/src/main/java/io/druid/guice/http/HttpClientModule.java b/server/src/main/java/io/druid/guice/http/HttpClientModule.java index c8dc0915ac8f..9bfda3e89a6f 100644 --- a/server/src/main/java/io/druid/guice/http/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/http/HttpClientModule.java @@ -27,9 +27,13 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.annotations.Global; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; import java.lang.annotation.Annotation; +import java.util.List; /** */ @@ -118,7 +122,24 @@ public HttpClient get() builder.withSslContext(getSslContextBinding().getProvider().get()); } - return HttpClientInit.createClient(builder.build(), LifecycleUtils.asMmxLifecycle(getLifecycleProvider().get())); + HttpClient client = HttpClientInit.createClient( + builder.build(), + LifecycleUtils.asMmxLifecycle(getLifecycleProvider().get()) + ); + final AuthConfig authConfig = getAuthConfig(); + if (authConfig.isEnabled()) { + List authenticators = getAuthenticatorChain(); + for (Authenticator authenticator : authenticators) { + if (authenticator.getTypeName().equals(authConfig.getInternalAuthenticator())) { + return authenticator.createInternalClient(client); + } + } + throw new ISE( + "Could not locate internal authenticator with type name: %s", + authConfig.getInternalAuthenticator() + ); + } + return client; } } } diff --git a/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java b/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java new file mode 100644 index 000000000000..6a2e7f81b317 --- /dev/null +++ b/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java @@ -0,0 +1,53 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.guice.security; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.multibindings.MapBinder; +import com.google.inject.name.Named; +import io.druid.guice.LazySingleton; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.PolyBind; +import io.druid.server.security.Authenticator; +import io.druid.server.security.NoopAuthenticator; + +public class AuthenticatorModule implements Module +{ + @Override + public void configure(Binder binder) + { + final MapBinder authenticatorMapBinder = PolyBind.optionBinder( + binder, + Key.get(Authenticator.class) + ); + authenticatorMapBinder.addBinding("noop").to(NoopAuthenticator.class).in(LazySingleton.class); + } + + @Provides + @ManageLifecycle + @Named("noop") + public Authenticator getAuthenticator() + { + return new NoopAuthenticator(); + } +} diff --git a/server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java b/server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java new file mode 100644 index 000000000000..3fe687fe45f3 --- /dev/null +++ b/server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java @@ -0,0 +1,53 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.guice.security; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.multibindings.MapBinder; +import com.google.inject.name.Named; +import io.druid.guice.LazySingleton; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.PolyBind; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.NoopAuthorizationManager; + +public class AuthorizationManagerModule implements Module +{ + @Override + public void configure(Binder binder) + { + final MapBinder authorizationManagerMapBinder = PolyBind.optionBinder( + binder, + Key.get(AuthorizationManager.class) + ); + authorizationManagerMapBinder.addBinding("noop").to(NoopAuthorizationManager.class).in(LazySingleton.class); + } + + @Provides + @ManageLifecycle + @Named("noop") + public AuthorizationManager getAuthorizationManager() + { + return new NoopAuthorizationManager(); + } +} diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 981341bd7362..439055c1b96f 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -56,10 +56,13 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.guice.http.HttpClientModule; +import io.druid.guice.security.AuthenticatorModule; +import io.druid.guice.security.AuthorizationManagerModule; import io.druid.guice.security.DruidAuthModule; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; +import io.druid.server.initialization.AuthorizationManagerMapperModule; import io.druid.server.initialization.EmitterModule; import io.druid.server.initialization.jetty.JettyServerModule; import io.druid.server.metrics.MetricsModule; @@ -344,6 +347,9 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter // New modules should be added after Log4jShutterDownerModule new Log4jShutterDownerModule(), new DruidAuthModule(), + new AuthenticatorModule(), + new AuthorizationManagerModule(), + new AuthorizationManagerMapperModule(), new LifecycleModule(), EmitterModule.class, HttpClientModule.global(), diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index 402e59f4356c..cea6b6a94799 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -27,9 +27,11 @@ import io.druid.client.TimelineServerView; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.server.http.security.StateResourceFilter; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -58,6 +60,8 @@ public BrokerQueryResource( @Smile ObjectMapper smileMapper, QueryManager queryManager, AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper, + GenericQueryMetricsFactory queryMetricsFactory, TimelineServerView brokerServerView ) { @@ -66,7 +70,9 @@ public BrokerQueryResource( jsonMapper, smileMapper, queryManager, - authConfig + authConfig, + authorizationManagerMapper, + queryMetricsFactory ); this.brokerServerView = brokerServerView; } diff --git a/server/src/main/java/io/druid/server/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java index b32091960c93..da29ff954049 100644 --- a/server/src/main/java/io/druid/server/ClientInfoResource.java +++ b/server/src/main/java/io/druid/server/ClientInfoResource.java @@ -19,8 +19,6 @@ package io.druid.server; -import com.google.common.base.Predicate; -import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -35,18 +33,14 @@ import io.druid.client.TimelineServerView; import io.druid.client.selector.ServerSelector; import io.druid.common.utils.JodaUtils; -import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.query.LocatedSegmentDescriptor; import io.druid.query.TableDataSource; import io.druid.query.metadata.SegmentMetadataQueryConfig; import io.druid.server.http.security.DatasourceResourceFilter; -import io.druid.server.security.Access; -import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -66,7 +60,6 @@ import java.io.IOException; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -86,13 +79,15 @@ public class ClientInfoResource private TimelineServerView timelineServerView; private SegmentMetadataQueryConfig segmentMetadataQueryConfig; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; @Inject public ClientInfoResource( FilteredServerInventoryView serverInventoryView, TimelineServerView timelineServerView, SegmentMetadataQueryConfig segmentMetadataQueryConfig, - AuthConfig authConfig + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper ) { this.serverInventoryView = serverInventoryView; @@ -100,6 +95,7 @@ public ClientInfoResource( this.segmentMetadataQueryConfig = (segmentMetadataQueryConfig == null) ? new SegmentMetadataQueryConfig() : segmentMetadataQueryConfig; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } private Map> getSegmentsForDatasources() @@ -122,28 +118,11 @@ private Map> getSegmentsForDatasources() public Iterable getDataSources(@Context final HttpServletRequest request) { if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final Map, Access> resourceAccessMap = new HashMap<>(); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - return Collections2.filter( + return AuthorizationUtils.filterAuthorizedResources( + request, getSegmentsForDatasources().keySet(), - new Predicate() - { - @Override - public boolean apply(String input) - { - Resource resource = new Resource(input, ResourceType.DATASOURCE); - Action action = Action.READ; - Pair key = new Pair<>(resource, action); - if (resourceAccessMap.containsKey(key)) { - return resourceAccessMap.get(key).isAllowed(); - } else { - Access access = authorizationInfo.isAuthorized(key.lhs, key.rhs); - resourceAccessMap.put(key, access); - return access.isAllowed(); - } - } - } + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizationManagerMapper ); } else { return getSegmentsForDatasources().keySet(); diff --git a/server/src/main/java/io/druid/server/QueryLifecycle.java b/server/src/main/java/io/druid/server/QueryLifecycle.java index 4104e1638e25..c913f5970a3d 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycle.java +++ b/server/src/main/java/io/druid/server/QueryLifecycle.java @@ -39,14 +39,13 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.security.Access; -import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import org.joda.time.DateTime; import javax.annotation.Nullable; +import javax.servlet.http.HttpServletRequest; import java.util.LinkedHashMap; import java.util.Map; import java.util.UUID; @@ -76,6 +75,7 @@ public class QueryLifecycle private final RequestLogger requestLogger; private final ServerConfig serverConfig; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; private final long startMs; private final long startNs; @@ -91,6 +91,7 @@ public QueryLifecycle( final RequestLogger requestLogger, final ServerConfig serverConfig, final AuthConfig authConfig, + final AuthorizationManagerMapper authorizationManagerMapper, final long startMs, final long startNs ) @@ -102,6 +103,7 @@ public QueryLifecycle( this.requestLogger = requestLogger; this.serverConfig = serverConfig; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; this.startMs = startMs; this.startNs = startNs; } @@ -115,14 +117,19 @@ public QueryLifecycle( * @param authorizationInfo authorization info from the request; or null if none is present. This must be non-null * if security is enabled, or the request will be considered unauthorized. * @param remoteAddress remote address, for logging; or null if unknown + * @param needsAuth if false, skip the authorization check. This is useful when the authorization check has + * already been performed (e.g. in SQL handling, where authorization takes place in the + * planning step) * * @return results */ @SuppressWarnings("unchecked") public Sequence runSimple( final Query query, - @Nullable final AuthorizationInfo authorizationInfo, - @Nullable final String remoteAddress + @Nullable final String user, + @Nullable final String namespace, + @Nullable final String remoteAddress, + boolean needsAuth ) { initialize(query); @@ -130,9 +137,13 @@ public Sequence runSimple( final Sequence results; try { - final Access access = authorize(authorizationInfo); - if (!access.isAllowed()) { - throw new ISE("Unauthorized"); + if (needsAuth) { + final Access access = authorize(user, namespace, null); + if (!access.isAllowed()) { + throw new ISE("Unauthorized"); + } + } else { + transition(State.INITIALIZED, State.AUTHORIZED); } final QueryLifecycle.QueryResponse queryResponse = execute(); @@ -183,37 +194,48 @@ public void initialize(final Query baseQuery) /** * Authorize the query. Will return an Access object denoting whether the query is authorized or not. * - * @param authorizationInfo authorization info from the request; or null if none is present. This must be non-null - * if security is enabled, or the request will be considered unauthorized. + * @param token authentication token from the request + * @param namespace namespace of the authentication token + * @param req HTTP request object of the request. If provided, the auth-related fields in the HTTP request + * will be automatically set. * * @return authorization result * - * @throws IllegalStateException if security is enabled and authorizationInfo is null - */ - public Access authorize(@Nullable final AuthorizationInfo authorizationInfo) + * */ + public Access authorize( + @Nullable final String token, + @Nullable final String namespace, + @Nullable HttpServletRequest req + ) { transition(State.INITIALIZED, State.AUTHORIZING); if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - if (authorizationInfo != null) { - for (String dataSource : queryPlus.getQuery().getDataSource().getNames()) { - Access authResult = authorizationInfo.isAuthorized( - new Resource(dataSource, ResourceType.DATASOURCE), - Action.READ - ); - if (!authResult.isAllowed()) { - // Not authorized; go straight to Jail, do not pass Go. - transition(State.AUTHORIZING, State.DONE); - return authResult; - } - } + Access authResult; + if (req != null) { + authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + queryPlus.getQuery().getDataSource().getNames(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizationManagerMapper + ); + } else { + authResult = AuthorizationUtils.authorizeAllResourceActions( + queryPlus.getQuery().getDataSource().getNames(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + token, + namespace, + authorizationManagerMapper + ); + } - transition(State.AUTHORIZING, State.AUTHORIZED); - return new Access(true); + if (!authResult.isAllowed()) { + // Not authorized; go straight to Jail, do not pass Go. + transition(State.AUTHORIZING, State.DONE); } else { - throw new ISE("WTF?! Security is enabled but no authorization info found in the request"); + transition(State.AUTHORIZING, State.AUTHORIZED); } + return authResult; } else { transition(State.AUTHORIZING, State.AUTHORIZED); return new Access(true); diff --git a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java index a6fbc3cca719..921964efeea2 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java +++ b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java @@ -28,6 +28,7 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; @LazySingleton public class QueryLifecycleFactory @@ -39,6 +40,7 @@ public class QueryLifecycleFactory private final RequestLogger requestLogger; private final ServerConfig serverConfig; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; @Inject public QueryLifecycleFactory( @@ -48,8 +50,9 @@ public QueryLifecycleFactory( final ServiceEmitter emitter, final RequestLogger requestLogger, final ServerConfig serverConfig, - final AuthConfig authConfig - ) + final AuthConfig authConfig, + final AuthorizationManagerMapper authorizationManagerMapper + ) { this.warehouse = warehouse; this.texasRanger = texasRanger; @@ -58,6 +61,7 @@ public QueryLifecycleFactory( this.requestLogger = requestLogger; this.serverConfig = serverConfig; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } public QueryLifecycle factorize() @@ -70,6 +74,7 @@ public QueryLifecycle factorize() requestLogger, serverConfig, authConfig, + authorizationManagerMapper, System.currentTimeMillis(), System.nanoTime() ); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 84caf56ee70f..b133661b3c87 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -24,9 +24,9 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.datatype.joda.ser.DateTimeSerializer; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; import com.google.common.io.CountingOutputStream; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; @@ -38,16 +38,15 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.Yielders; +import io.druid.query.GenericQueryMetricsFactory; import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryInterruptedException; import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.security.Access; -import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import org.joda.time.DateTime; import javax.servlet.http.HttpServletRequest; @@ -92,6 +91,9 @@ public class QueryResource implements QueryCountStatsProvider protected final ObjectMapper serializeDateTimeAsLongSmileMapper; protected final QueryManager queryManager; protected final AuthConfig authConfig; + protected final AuthorizationManagerMapper authorizationManagerMapper; + + private final GenericQueryMetricsFactory queryMetricsFactory; private final AtomicLong successfulQueryCount = new AtomicLong(); private final AtomicLong failedQueryCount = new AtomicLong(); private final AtomicLong interruptedQueryCount = new AtomicLong(); @@ -102,7 +104,9 @@ public QueryResource( @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, QueryManager queryManager, - AuthConfig authConfig + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper, + GenericQueryMetricsFactory queryMetricsFactory ) { this.queryLifecycleFactory = queryLifecycleFactory; @@ -112,6 +116,8 @@ public QueryResource( this.serializeDateTimeAsLongSmileMapper = serializeDataTimeAsLong(smileMapper); this.queryManager = queryManager; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; + this.queryMetricsFactory = queryMetricsFactory; } @DELETE @@ -123,25 +129,21 @@ public Response getServer(@PathParam("id") String queryId, @Context final HttpSe log.debug("Received cancel request for query [%s]", queryId); } if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); Set datasources = queryManager.getQueryDatasources(queryId); if (datasources == null) { log.warn("QueryId [%s] not registered with QueryManager, cannot cancel", queryId); - } else { - for (String dataSource : datasources) { - Access authResult = authorizationInfo.isAuthorized( - new Resource(dataSource, ResourceType.DATASOURCE), - Action.WRITE - ); - if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); - } - } + datasources = Sets.newTreeSet(); + } + + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + datasources, + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizationManagerMapper + ); + + if (!authResult.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); } } queryManager.cancelQuery(queryId); @@ -174,7 +176,11 @@ public Response doPost( log.debug("Got query [%s]", query); } - final Access authResult = queryLifecycle.authorize((AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)); + final Access authResult = queryLifecycle.authorize( + (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), + (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE), + req + ); if (!authResult.isAllowed()) { return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); } diff --git a/server/src/main/java/io/druid/server/http/DatasourcesResource.java b/server/src/main/java/io/druid/server/http/DatasourcesResource.java index df2d725d7e9d..66798e1ee898 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -42,7 +42,7 @@ import io.druid.query.TableDataSource; import io.druid.server.http.security.DatasourceResourceFilter; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -80,19 +80,22 @@ public class DatasourcesResource private final MetadataSegmentManager databaseSegmentManager; private final IndexingServiceClient indexingServiceClient; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; @Inject public DatasourcesResource( CoordinatorServerView serverInventoryView, MetadataSegmentManager databaseSegmentManager, @Nullable IndexingServiceClient indexingServiceClient, - AuthConfig authConfig + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper ) { this.serverInventoryView = serverInventoryView; this.databaseSegmentManager = databaseSegmentManager; this.indexingServiceClient = indexingServiceClient; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } @GET @@ -107,7 +110,9 @@ public Response getQueryableDataSources( final Set datasources = authConfig.isEnabled() ? InventoryViewUtils.getSecuredDataSources( serverInventoryView, - (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) + authorizationManagerMapper, + (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), + (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) ) : InventoryViewUtils.getDataSources(serverInventoryView); diff --git a/server/src/main/java/io/druid/server/http/IntervalsResource.java b/server/src/main/java/io/druid/server/http/IntervalsResource.java index 630531252c2c..b7facd67a8d9 100644 --- a/server/src/main/java/io/druid/server/http/IntervalsResource.java +++ b/server/src/main/java/io/druid/server/http/IntervalsResource.java @@ -27,7 +27,7 @@ import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.guava.Comparators; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -51,15 +51,18 @@ public class IntervalsResource { private final InventoryView serverInventoryView; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; @Inject public IntervalsResource( InventoryView serverInventoryView, - AuthConfig authConfig + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper ) { this.serverInventoryView = serverInventoryView; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } @GET @@ -70,7 +73,9 @@ public Response getIntervals(@Context final HttpServletRequest req) final Set datasources = authConfig.isEnabled() ? InventoryViewUtils.getSecuredDataSources( serverInventoryView, - (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) + authorizationManagerMapper, + (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), + (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) ) : InventoryViewUtils.getDataSources(serverInventoryView); @@ -103,7 +108,9 @@ public Response getSpecificIntervals( final Set datasources = authConfig.isEnabled() ? InventoryViewUtils.getSecuredDataSources( serverInventoryView, - (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) + authorizationManagerMapper, + (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), + (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) ) : InventoryViewUtils.getDataSources(serverInventoryView); diff --git a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java index 521b175f5916..be410a51173c 100644 --- a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java +++ b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java @@ -33,7 +33,8 @@ import io.druid.java.util.common.Pair; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.server.security.Resource; import io.druid.server.security.ResourceType; @@ -80,11 +81,18 @@ public Iterable apply(DruidServer input) public static Set getSecuredDataSources( InventoryView inventoryView, - final AuthorizationInfo authorizationInfo + final AuthorizationManagerMapper authorizationManagerMapper, + final String identity, + final String namespace ) { - if (authorizationInfo == null) { - throw new ISE("Invalid to call a secured method with null AuthorizationInfo!!"); + if (authorizationManagerMapper == null) { + throw new ISE("No authorization mapper found"); + } + + final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); + if (authorizationManager == null) { + throw new ISE("Invalid to call a secured method with null AuthorizationManager!!"); } else { final Map, Access> resourceAccessMap = new HashMap<>(); return ImmutableSet.copyOf( @@ -101,7 +109,7 @@ public boolean apply(DruidDataSource input) if (resourceAccessMap.containsKey(key)) { return resourceAccessMap.get(key).isAllowed(); } else { - Access access = authorizationInfo.isAuthorized(key.lhs, key.rhs); + Access access = authorizationManager.authorize(identity, key.lhs, key.rhs); resourceAccessMap.put(key, access); return access.isAllowed(); } diff --git a/server/src/main/java/io/druid/server/http/MetadataResource.java b/server/src/main/java/io/druid/server/http/MetadataResource.java index fb60b2a0bf68..f6034fec1e20 100644 --- a/server/src/main/java/io/druid/server/http/MetadataResource.java +++ b/server/src/main/java/io/druid/server/http/MetadataResource.java @@ -22,22 +22,17 @@ import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.Collections2; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; import io.druid.client.DruidDataSource; import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import io.druid.java.util.common.Pair; import io.druid.metadata.MetadataSegmentManager; import io.druid.server.http.security.DatasourceResourceFilter; -import io.druid.server.security.Access; -import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -52,9 +47,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import java.io.IOException; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; /** @@ -65,17 +58,20 @@ public class MetadataResource private final MetadataSegmentManager metadataSegmentManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; @Inject public MetadataResource( MetadataSegmentManager metadataSegmentManager, IndexerMetadataStorageCoordinator metadataStorageCoordinator, - AuthConfig authConfig + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper ) { this.metadataSegmentManager = metadataSegmentManager; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } @GET @@ -109,31 +105,14 @@ public String apply(DruidDataSource input) final Set dataSourceNamesPostAuth; if (authConfig.isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final Map, Access> resourceAccessMap = new HashMap<>(); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - dataSourceNamesPostAuth = ImmutableSet.copyOf( - Sets.filter( - dataSourceNamesPreAuth, - new Predicate() - { - @Override - public boolean apply(String input) - { - Resource resource = new Resource(input, ResourceType.DATASOURCE); - Action action = Action.READ; - Pair key = new Pair<>(resource, action); - if (resourceAccessMap.containsKey(key)) { - return resourceAccessMap.get(key).isAllowed(); - } else { - Access access = authorizationInfo.isAuthorized(key.lhs, key.rhs); - resourceAccessMap.put(key, access); - return access.isAllowed(); - } - } - } - ) + List datasourceNamesList = AuthorizationUtils.filterAuthorizedResources( + req, + dataSourceNamesPreAuth, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizationManagerMapper ); + + dataSourceNamesPostAuth = Sets.newTreeSet(datasourceNamesList); } else { dataSourceNamesPostAuth = dataSourceNamesPreAuth; } diff --git a/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java b/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java index a8a1fb4cb4e1..19f372b40d4e 100644 --- a/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java @@ -26,6 +26,8 @@ import com.sun.jersey.spi.container.ResourceFilter; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; + import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Context; @@ -36,11 +38,16 @@ public abstract class AbstractResourceFilter implements ResourceFilter, Containe private HttpServletRequest req; private final AuthConfig authConfig; + private AuthorizationManagerMapper authorizationManagerMapper; @Inject - public AbstractResourceFilter(AuthConfig authConfig) + public AbstractResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } @Override @@ -65,6 +72,16 @@ public AuthConfig getAuthConfig() return authConfig; } + public AuthorizationManagerMapper getAuthorizationManagerMapper() + { + return authorizationManagerMapper; + } + + public void setAuthorizationManagerMapper(AuthorizationManagerMapper authorizationManagerMapper) + { + this.authorizationManagerMapper = authorizationManagerMapper; + } + public AbstractResourceFilter setReq(HttpServletRequest req) { this.req = req; diff --git a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java index 926df47f07b7..0baaedcc6181 100644 --- a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java @@ -19,14 +19,15 @@ package io.druid.server.http.security; -import com.google.common.base.Preconditions; import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import javax.ws.rs.WebApplicationException; @@ -44,27 +45,29 @@ public class ConfigResourceFilter extends AbstractResourceFilter { @Inject - public ConfigResourceFilter(AuthConfig authConfig) + public ConfigResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { - super(authConfig); + super(authConfig, authorizationManagerMapper); } @Override public ContainerRequest filter(ContainerRequest request) { if (getAuthConfig().isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final String resourceName = "CONFIG"; - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" + final ResourceAction resourceAction = new ResourceAction( + new Resource("CONFIG", ResourceType.CONFIG), + getAction(request) ); - final Access authResult = authorizationInfo.isAuthorized( - new Resource(resourceName, ResourceType.CONFIG), - getAction(request) + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizationManagerMapper() ); + if (!authResult.isAllowed()) { throw new WebApplicationException( Response.status(Response.Status.FORBIDDEN) diff --git a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java index 073214ba3935..f28390440657 100644 --- a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java @@ -28,8 +28,10 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import javax.ws.rs.WebApplicationException; @@ -47,40 +49,29 @@ public class DatasourceResourceFilter extends AbstractResourceFilter { @Inject - public DatasourceResourceFilter(AuthConfig authConfig) + public DatasourceResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { - super(authConfig); + super(authConfig, authorizationManagerMapper); } @Override public ContainerRequest filter(ContainerRequest request) { if (getAuthConfig().isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final String dataSourceName = request.getPathSegments() - .get( - Iterables.indexOf( - request.getPathSegments(), - new Predicate() - { - @Override - public boolean apply(PathSegment input) - { - return input.getPath().equals("datasources"); - } - } - ) + 1 - ).getPath(); - Preconditions.checkNotNull(dataSourceName); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); - final Access authResult = authorizationInfo.isAuthorized( - new Resource(dataSourceName, ResourceType.DATASOURCE), + final ResourceAction resourceAction = new ResourceAction( + new Resource(getRequestDatasourceName(request), ResourceType.DATASOURCE), getAction(request) ); + + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizationManagerMapper() + ); + if (!authResult.isAllowed()) { throw new WebApplicationException( Response.status(Response.Status.FORBIDDEN) @@ -93,6 +84,26 @@ public boolean apply(PathSegment input) return request; } + private String getRequestDatasourceName(ContainerRequest request) + { + final String dataSourceName = request.getPathSegments() + .get( + Iterables.indexOf( + request.getPathSegments(), + new Predicate() + { + @Override + public boolean apply(PathSegment input) + { + return input.getPath().equals("datasources"); + } + } + ) + 1 + ).getPath(); + Preconditions.checkNotNull(dataSourceName); + return dataSourceName; + } + @Override public boolean isApplicable(String requestPath) { diff --git a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java index 1f73bd3b9849..221f62d5433e 100644 --- a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java @@ -28,8 +28,10 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import javax.ws.rs.WebApplicationException; @@ -47,16 +49,18 @@ public class RulesResourceFilter extends AbstractResourceFilter { @Inject - public RulesResourceFilter(AuthConfig authConfig) + public RulesResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { - super(authConfig); + super(authConfig, authorizationManagerMapper); } @Override public ContainerRequest filter(ContainerRequest request) { if (getAuthConfig().isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 final String dataSourceName = request.getPathSegments() .get( Iterables.indexOf( @@ -72,15 +76,18 @@ public boolean apply(PathSegment input) ) + 1 ).getPath(); Preconditions.checkNotNull(dataSourceName); - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" - ); - final Access authResult = authorizationInfo.isAuthorized( + + final ResourceAction resourceAction = new ResourceAction( new Resource(dataSourceName, ResourceType.DATASOURCE), getAction(request) ); + + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizationManagerMapper() + ); + if (!authResult.isAllowed()) { throw new WebApplicationException( Response.status(Response.Status.FORBIDDEN) diff --git a/server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java b/server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java new file mode 100644 index 000000000000..65d06fe31d5b --- /dev/null +++ b/server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java @@ -0,0 +1,85 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.http.security; + +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; +import com.sun.jersey.spi.container.ContainerRequest; +import io.druid.server.security.Access; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; +import io.druid.server.security.ResourceType; + +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; +import java.util.List; + +public class SecurityResourceFilter extends AbstractResourceFilter +{ + @Inject + public SecurityResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) + { + super(authConfig, authorizationManagerMapper); + } + + @Override + public ContainerRequest filter(ContainerRequest request) + { + if (getAuthConfig().isEnabled()) { + final ResourceAction resourceAction = new ResourceAction( + new Resource("security", ResourceType.CONFIG), + getAction(request) + ); + + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizationManagerMapper() + ); + + if (!authResult.isAllowed()) { + throw new WebApplicationException( + Response.status(Response.Status.FORBIDDEN) + .entity(String.format("Access-Check-Result: %s", authResult.toString())) + .build() + ); + } + } + return request; + } + + @Override + public boolean isApplicable(String requestPath) + { + List applicablePaths = ImmutableList.of("druid/coordinator/v1/security/"); + for (String path : applicablePaths) { + if(requestPath.startsWith(path) && !requestPath.equals(path)) { + return true; + } + } + return false; + } +} diff --git a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java index cec7ecd21c2c..7fafc9877f33 100644 --- a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java @@ -19,14 +19,15 @@ package io.druid.server.http.security; -import com.google.common.base.Preconditions; import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; import javax.ws.rs.WebApplicationException; @@ -50,27 +51,29 @@ public class StateResourceFilter extends AbstractResourceFilter { @Inject - public StateResourceFilter(AuthConfig authConfig) + public StateResourceFilter( + AuthConfig authConfig, + AuthorizationManagerMapper authorizationManagerMapper + ) { - super(authConfig); + super(authConfig, authorizationManagerMapper); } @Override public ContainerRequest filter(ContainerRequest request) { if (getAuthConfig().isEnabled()) { - // This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424 - final String resourceName = "STATE"; - final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - Preconditions.checkNotNull( - authorizationInfo, - "Security is enabled but no authorization info found in the request" + final ResourceAction resourceAction = new ResourceAction( + new Resource("STATE", ResourceType.STATE), + getAction(request) ); - final Access authResult = authorizationInfo.isAuthorized( - new Resource(resourceName, ResourceType.STATE), - getAction(request) + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizationManagerMapper() ); + if (!authResult.isAllowed()) { throw new WebApplicationException( Response.status(Response.Status.FORBIDDEN) diff --git a/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java new file mode 100644 index 000000000000..e8df9d987bb1 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java @@ -0,0 +1,102 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.initialization; + +import com.fasterxml.jackson.databind.Module; +import com.google.common.collect.Maps; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.name.Names; +import io.druid.guice.LazySingleton; +import io.druid.initialization.DruidModule; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.DefaultAuthorizationManager; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class AuthorizationManagerMapperModule implements DruidModule +{ + private static Logger log = new Logger(AuthorizationManagerMapperModule.class); + + @Override + public void configure(Binder binder) + { + binder.bind(AuthorizationManagerMapper.class) + .toProvider(new AuthorizationManagerMapperProvider()) + .in(LazySingleton.class); + } + + @Override + public List getJacksonModules() + { + return Collections.EMPTY_LIST; + } + + private static class AuthorizationManagerMapperProvider implements Provider + { + private AuthConfig authConfig; + private Injector injector; + + @Inject + public void inject(Injector injector) + { + this.authConfig = injector.getInstance(AuthConfig.class); + this.injector = injector; + } + + @Override + public AuthorizationManagerMapper get() + { + Map authorizationManagerMap = Maps.newHashMap(); + + List authManagerNames = authConfig.getAuthorizationManagers(); + + // If user didn't configure any AuthorizationManagers, use the default which rejects all requests. + if (authManagerNames == null || authManagerNames.size() == 0) { + return new AuthorizationManagerMapper(null) { + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new DefaultAuthorizationManager(); + } + }; + } + + for (String authorizationManagerName : authManagerNames) { + AuthorizationManager authorizationManager = injector.getInstance(Key.get( + AuthorizationManager.class, + Names.named(authorizationManagerName) + )); + + authorizationManagerMap.put(authorizationManager.getNamespace(), authorizationManager); + } + + return new AuthorizationManagerMapper(authorizationManagerMap); + } + } +} diff --git a/server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java b/server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java new file mode 100644 index 000000000000..254c92ce5e4f --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java @@ -0,0 +1,46 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.initialization; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import io.druid.guice.ManageLifecycle; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.NoopAuthorizationManager; + +public class NoopAuthorizationManagerModule implements Module +{ + public static final String TYPE = "noop"; + + @Override + public void configure(Binder binder) + { + } + + @Provides + @ManageLifecycle + @Named(TYPE) + public AuthorizationManager makeAuthorizationManager() + { + return new NoopAuthorizationManager(); + } +} diff --git a/server/src/main/java/io/druid/server/security/Action.java b/server/src/main/java/io/druid/server/security/Action.java index 2b7606b58dd8..68a038c362d3 100644 --- a/server/src/main/java/io/druid/server/security/Action.java +++ b/server/src/main/java/io/druid/server/security/Action.java @@ -19,8 +19,19 @@ package io.druid.server.security; +import com.fasterxml.jackson.annotation.JsonCreator; + public enum Action { READ, - WRITE + WRITE; + + @JsonCreator + public static Action fromString(String name) + { + if (name == null) { + return null; + } + return valueOf(name.toUpperCase()); + } } diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index 619ab158a753..f9ebf95bc550 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -22,37 +22,94 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + public class AuthConfig { /** * Use this String as the attribute name for the request attribute to pass {@link AuthorizationInfo} * from the servlet filter to the jersey resource - * */ + */ public static final String DRUID_AUTH_TOKEN = "Druid-Auth-Token"; + /** + * HTTP attribute set when a static method in AuthorizationUtils performs an authorization check on the request. + */ + public static final String DRUID_AUTH_TOKEN_CHECKED = "Druid-Auth-Token-Checked"; + + /** + * HTTP attribute that indicates the namespace for a request. Set by Authenticator implementations when + * they successfully authenticate a request. The AuthorizationManager with a matching namespace will be used to + * authorize the request. + */ + public static final String DRUID_AUTH_NAMESPACE = "Druid-Auth-Namespace"; + public AuthConfig() { - this(false); + this(false, null, null, null); } @JsonCreator - public AuthConfig(@JsonProperty("enabled") boolean enabled) + public AuthConfig( + @JsonProperty("enabled") boolean enabled, + @JsonProperty("authenticatorChain") List authenticationChain, + @JsonProperty("internalAuthenticator") String internalAuthenticator, + @JsonProperty("authorizationManagers") List authorizationManagers + ) { this.enabled = enabled; + this.authenticatorChain = authenticationChain; + this.internalAuthenticator = internalAuthenticator; + this.authorizationManagers = authorizationManagers; } + /** * If druid.auth.enabled is set to true then an implementation of AuthorizationInfo * must be provided and it must be set as a request attribute possibly inside the servlet filter * injected in the filter chain using your own extension - * */ + */ @JsonProperty private final boolean enabled; + @JsonProperty + private final List authenticatorChain; + + @JsonProperty + private final String internalAuthenticator; + + @JsonProperty + List authorizationManagers; + public boolean isEnabled() { return enabled; } + public List getAuthenticatorChain() + { + return authenticatorChain; + } + + public String getInternalAuthenticator() + { + return internalAuthenticator; + } + + public List getAuthorizationManagers() + { + return authorizationManagers; + } + + @Override + public String toString() + { + return "AuthConfig{" + + "enabled=" + enabled + + ", authenticatorChain='" + authenticatorChain + '\'' + + ", internalAuthenticator='" + internalAuthenticator + '\'' + + '}'; + } + @Override public boolean equals(Object o) { @@ -65,21 +122,22 @@ public boolean equals(Object o) AuthConfig that = (AuthConfig) o; - return enabled == that.enabled; + if (isEnabled() != that.isEnabled()) { + return false; + } + if (!getInternalAuthenticator().equals(that.getInternalAuthenticator())) { + return false; + } + return getAuthenticatorChain().equals(that.getAuthenticatorChain()); } @Override public int hashCode() { - return (enabled ? 1 : 0); - } - - @Override - public String toString() - { - return "AuthConfig{" + - "enabled=" + enabled + - '}'; + int result = (isEnabled() ? 1 : 0); + result = 31 * result + getAuthenticatorChain().hashCode(); + result = 31 * result + getInternalAuthenticator().hashCode(); + return result; } } diff --git a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java new file mode 100644 index 000000000000..86cce524dd16 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java @@ -0,0 +1,105 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.name.Names; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.ServletContextHandler; + +import java.util.List; + +public class AuthenticationUtils +{ + public static List getAuthenticatorChainFromConfig( + List authenticatorChain, + Injector injector + ) + { + Preconditions.checkNotNull(authenticatorChain, "No authenticator chain defined!"); + List authenticators = Lists.newArrayList(); + for (String authenticatorName : authenticatorChain) { + authenticators.add( + injector.getInstance(Key.get(Authenticator.class, Names.named(authenticatorName))) + ); + } + + return authenticators; + } + + public static void addAuthenticationFilterChain( + ServletContextHandler root, + List authenticators + ) + { + for (Authenticator authenticator : authenticators) { + FilterHolder holder = new FilterHolder(authenticator.getFilter()); + if (authenticator.getInitParameters() != null) { + holder.setInitParameters(authenticator.getInitParameters()); + } + root.addFilter( + holder, + "/*", + null + ); + } + } + + public static void addNoopAuthorizationFilters(ServletContextHandler root, List unsecuredPaths) + { + for (String unsecuredPath : unsecuredPaths) { + root.addFilter(new FilterHolder(new UnsecuredResourceFilter()), unsecuredPath, null); + } + } + + public static void addSecuritySanityCheckFilter( + ServletContextHandler root, + ObjectMapper jsonMapper + ) + { + root.addFilter( + new FilterHolder( + new SecuritySanityCheckFilter(jsonMapper) + ), + "/*", + null + ); + } + + public static void addPreResponseAuthorizationCheckFilter( + ServletContextHandler root, + List authenticators, + ObjectMapper jsonMapper, + AuthConfig authConfig + ) + { + root.addFilter( + new FilterHolder( + new PreResponseAuthorizationCheckFilter(authConfig, authenticators, jsonMapper) + ), + "/*", + null + ); + } +} diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java new file mode 100644 index 000000000000..d27c351ddbb3 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -0,0 +1,101 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.metamx.http.client.HttpClient; +import io.druid.server.initialization.jetty.ServletFilterHolder; + +import javax.servlet.Filter; +import java.util.Map; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "noop", value = NoopAuthenticator.class), +}) +public interface Authenticator extends ServletFilterHolder +{ + /** + * @return The type name of this authenticator. Should be identical to the JsonTypeInfo type. + */ + public String getTypeName(); + + /** + * @return The namespace associated with this Authenticator. This will be used for choosing the correct + * AuthorizationManager for authorizing requests that have been authenticated by this Authenticator. + */ + public String getNamespace(); + + /** + * Create a Filter that performs authentication checks on incoming HTTP requests. + * + * If the authentication succeeds, the Filter should set the "Druid-Auth-Token" attribute in the request, + * containing a String that represents the authenticated identity of the requester. + * + * If the "Druid-Auth-Token" attribute is already set (i.e., request has been authenticated by an earlier Filter), + * this Filter should skip any authentication checks and proceed to the next Filter. + * + * If the authentication fails, the Filter should not send an error response. The error response will be sent + * after all Filters in the authentication filter chain have been checked. + * + * If an anonymous request is received, the Filter should continue on to the next Filter, the challenge response + * will be sent after the filter chain is exhausted. + * + * @return Filter that authenticates HTTP requests + */ + @Override + public Filter getFilter(); + + /** + * Return a WWW-Authenticate challenge scheme string appropriate for this Authenticator's authentication mechanism. + * + * For example, a Basic HTTP implementation should return "Basic", while a Kerberos implementation would return + * "Negotiate". + * + * @return Authentication scheme + */ + public String getAuthChallengeHeader(); + + /** + * Given a JDBC connection context, authenticate the identity represented by the information in the context. + * This is used to secure JDBC access for Druid SQL. + * + * For example, a Basic HTTP auth implementation could read the "user" and "password" fields from the JDBC context. + * + * The expected contents of the context are left to the implementation. + * + * @param context JDBC connection context + * @return true if the identity represented by the context is successfully authenticated + */ + public boolean authenticateJDBCContext(Map context); + + /** + * Return a client that sends requests with the format/information necessary to authenticate successfully + * against this Authenticator's authentication scheme using the identity of the internal system user. + * + * This HTTP client is used for internal communications between Druid nodes, such as when a broker communicates + * with a historical node during query processing. + * + * @param baseClient Base HTTP client for internal Druid communications + * @return HttpClient that sends requests with the credentials of the internal system user + */ + public HttpClient createInternalClient(HttpClient baseClient); +} diff --git a/server/src/main/java/io/druid/server/security/AuthorizationInfo.java b/server/src/main/java/io/druid/server/security/AuthorizationInfo.java deleted file mode 100644 index 31097a935477..000000000000 --- a/server/src/main/java/io/druid/server/security/AuthorizationInfo.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.server.security; - -/** - * This interface should be used to store as well as process Authorization Information - * An extension can be used to inject servlet filter which will create objects of this type - * and set it as a request attribute with attribute name as {@link AuthConfig#DRUID_AUTH_TOKEN}. - * In the jersey resources if the authorization is enabled depending on {@link AuthConfig#enabled} - * the {@link #isAuthorized(Resource, Action)} method will be used to perform authorization checks - * */ -public interface AuthorizationInfo -{ - /** - * Perform authorization checks for the given {@link Resource} and {@link Action}. - * resource and action objects should be instantiated depending on - * the specific endPoint where the check is being performed. - * Modeling Principal and specific way of performing authorization checks is - * entirely implementation dependent. - * - * @param resource information about resource that is being accessed - * @param action action to be performed on the resource - * @return a {@link Access} object having {@link Access#allowed} set to true if authorized otherwise set to false - * and optionally {@link Access#message} set to appropriate message - * */ - Access isAuthorized(Resource resource, Action action); -} diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManager.java b/server/src/main/java/io/druid/server/security/AuthorizationManager.java new file mode 100644 index 000000000000..ae5be80debfb --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthorizationManager.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultAuthorizationManager.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "default", value = DefaultAuthorizationManager.class), + @JsonSubTypes.Type(name = "noop", value = NoopAuthorizationManager.class) +}) +public interface AuthorizationManager +{ + /** + * Check if the entity represented by `identity` in `namespace` is authorized to perform `action` on `resource`. + * + * @param identity The identity of the requester + * @param namespace The namespace of the identity + * @param resource The resource to be accessed + * @param action The action to perform on the resource + * @return An Access object representing the result of the authorization check. + */ + public Access authorize(String identity, Resource resource, Action action); + + /** + * @return The namespace associated with this AuthorizationManager. Authenticator implementations will + * put the namespace in request headers. + */ + public String getNamespace(); +} diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java b/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java new file mode 100644 index 000000000000..cc4440750829 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java @@ -0,0 +1,39 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import java.util.Map; + +public class AuthorizationManagerMapper +{ + private Map authorizationManagerMap; + + public AuthorizationManagerMapper( + Map authorizationManagerMap + ) + { + this.authorizationManagerMap = authorizationManagerMap; + } + + public AuthorizationManager getAuthorizationManager(String namespace) + { + return authorizationManagerMap.get(namespace); + } +} diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java new file mode 100644 index 000000000000..6091195ea83b --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -0,0 +1,346 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import io.druid.java.util.common.ISE; + +import javax.servlet.http.HttpServletRequest; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Static utility functions for performing authorization checks. + */ +public class AuthorizationUtils +{ + public final static Access ACCESS_OK = new Access(true, "All resource-actions authorized."); + + /** + * Check a resource-action using the AuthorizationInfo from the request. + * + * Otherwise, if the resource-actions is authorized, return ACCESS_OK. + * + * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * + * If this attribute is already set when this function is called, an exception is thrown. + * + * @param request HTTP request to be authorized + * @param resourceAction A resource identifier and the action to be taken the resource. + * @return ACCESS_OK or the failed Access object returned by the request's AuthorizationInfo. + */ + public static Access authorizeResourceAction( + final HttpServletRequest request, + final ResourceAction resourceAction, + final AuthorizationManagerMapper authorizationManagerMapper + ) + { + return authorizeAllResourceActions( + request, + Lists.newArrayList(resourceAction), + authorizationManagerMapper + ); + } + + + /** + * Check a list of resource-actions using the AuthorizationInfo from the request. + * + * If one of the resource-actions fails the authorization check, this method returns the failed + * Access object from the check. + * + * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. + * + * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * + * If this attribute is already set when this function is called, an exception is thrown. + * @param request HTTP request to be authorized + * @param resourceActions A list of resource-actions to authorize + * @return ACCESS_OK or the Access object from the first failed check + */ + public static Access authorizeAllResourceActions( + final HttpServletRequest request, + final List resourceActions, + final AuthorizationManagerMapper authorizationManagerMapper + ) + { + final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); + if (identity == null) { + throw new ISE("Null identity."); + } + + final String namespace = (String) request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE); + if (namespace == null) { + throw new ISE("Null namespace."); + } + + final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); + if (authorizationManager == null) { + throw new ISE("No authorization manager found for namespace: [%s].", namespace); + } + + for (ResourceAction resourceAction : resourceActions) { + final Access access = authorizationManager.authorize( + identity, + resourceAction.getResource(), + resourceAction.getAction() + ); + if (!access.isAllowed()) { + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + return access; + } + } + + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + return ACCESS_OK; + } + + + /** + * Check a list of caller-defined resources, after converting them into a list of resource-actions + * using a caller provided function. + * + * If one of the resource-actions fails the authorization check, this method returns the failed + * Access object from the check. + * + * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. + * + * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * + * If this attribute is already set when this fImmutableList.>of(SupervisorManager.class, AuthorizationManager.class)unction is called, an exception is thrown. + * + * @param request HTTP request to be generated + * @param resources List of resources + * @param raGenerator Function that creates a resource-action from a resource + * @param Type of the resources in the resource list + * @return ACCESS_OK or the Access object from the first failed check + */ + public static Access authorizeAllResourceActions( + final HttpServletRequest request, + final Collection resources, + final Function raGenerator, + final AuthorizationManagerMapper authorizationManagerMapper + ) + { + final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); + if (identity == null) { + throw new ISE("Null identity."); + } + + final String namespace = (String) request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE); + if (namespace == null) { + throw new ISE("Null namespace."); + } + + final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); + if (authorizationManager == null) { + throw new ISE("No authorization manager found for namespace: [%s].", namespace); + } + + for (ResType resource : resources) { + final ResourceAction resourceAction = raGenerator.apply(resource); + final Access access = authorizationManager.authorize( + identity, + resourceAction.getResource(), + resourceAction.getAction() + ); + if (!access.isAllowed()) { + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + return access; + } + } + + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + return ACCESS_OK; + } + + /** + * Check a list of caller-defined resources, after converting them into a list of resource-actions + * using a caller provided function. + * + * If one of the resource-actions fails the authorization check, this method returns the failed + * Access object from the check. + * + * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. + * + * @param resources List of resources + * @param raGenerator Function that creates a resource-action from a resource + * @param Type of the resources in the resource list + * @return ACCESS_OK or the Access object from the first failed check + */ + public static Access authorizeAllResourceActions( + final Collection resources, + final Function raGenerator, + final String user, + final String namespace, + final AuthorizationManagerMapper authorizationManagerMapper + ) + { + if (user == null || namespace == null) { + throw new ISE("null user or namespace"); + } + final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); + if (authorizationManager == null) { + throw new ISE("No authorization manager found for namespace: [%s].", namespace); + } + + for (ResType resource : resources) { + final ResourceAction resourceAction = raGenerator.apply(resource); + final Access access = authorizationManager.authorize( + user, + resourceAction.getResource(), + resourceAction.getAction() + ); + if (!access.isAllowed()) { + return access; + } + } + + return ACCESS_OK; + } + + /** + * Check a list of resource-actions using the AuthorizationInfo from the request. + * + * If one of the resource-actions fails the authorization check, this method returns the failed + * Access object from the check. + * + * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. + * + * @param resourceActions A list of resource-actions to authorize + * @return ACCESS_OK or the Access object from the first failed check + */ + public static Access authorizeAllResourceActions( + final String user, + final String namespace, + final AuthorizationManagerMapper authorizationManagerMapper, + final List resourceActions + ) + { + if (user == null || namespace == null) { + throw new ISE("null user or namespace"); + } + final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); + if (authorizationManager == null) { + throw new ISE("No authorization manager found for namespace: [%s].", namespace); + } + + for (ResourceAction resourceAction : resourceActions) { + final Access access = authorizationManager.authorize( + user, + resourceAction.getResource(), + resourceAction.getAction() + ); + if (!access.isAllowed()) { + return access; + } + } + + return ACCESS_OK; + } + + /** + * Filter a list of resource-actions using the request's AuthorizationInfo, returning a new list of + * resource-actions that were authorized. + * + * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * + * If this attribute is already set when this function is called, an exception is thrown. + * + * @param request HTTP request to be authorized + * @param resources List of resources to be processed into resource-actions + * @param resourceActionGenerator Function that creates a resource-action from a resource + * @return A list containing the resource-actions from the resourceParser that were successfully authorized. + */ + + public static List filterAuthorizedResources( + final HttpServletRequest request, + final Collection resources, + final Function resourceActionGenerator, + final AuthorizationManagerMapper authorizationManagerMapper + ) + { + final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); + if (identity == null) { + throw new ISE("Null identity."); + } + + final String namespace = (String) request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE); + if (namespace == null) { + throw new ISE("Null namespace."); + } + + final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); + if (authorizationManager == null) { + throw new ISE("No authorization manager found for namespace: [%s].", namespace); + } + + List filteredResources = new ArrayList<>(); + for (ResType resource : resources) { + final ResourceAction resourceAction = resourceActionGenerator.apply(resource); + final Access access = authorizationManager.authorize( + identity, + resourceAction.getResource(), + resourceAction.getAction() + ); + if (access.isAllowed()) { + filteredResources.add(resource); + } + } + + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, filteredResources.size() > 0); + return filteredResources; + } + + /** + * Function for the common pattern of generating a resource-action for reading from a datasource, using the + * datasource name. + */ + public static Function DATASOURCE_READ_RA_GENERATOR = new Function() + { + @Override + public ResourceAction apply(String input) + { + return new ResourceAction( + new Resource(input, ResourceType.DATASOURCE), + Action.READ + ); + } + }; + + + /** + * Function for the common pattern of generating a resource-action for reading from a datasource, using the + * datasource name. + */ + public static Function DATASOURCE_WRITE_RA_GENERATOR = new Function() + { + @Override + public ResourceAction apply(String input) + { + return new ResourceAction( + new Resource(input, ResourceType.DATASOURCE), + Action.WRITE + ); + } + }; +} diff --git a/server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java b/server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java new file mode 100644 index 000000000000..7169e121d2a1 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java @@ -0,0 +1,35 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +public class DefaultAuthorizationManager implements AuthorizationManager +{ + @Override + public Access authorize(String identity, Resource resource, Action action) + { + return new Access(false, "Please configure a non-default Authorization Manager."); + } + + @Override + public String getNamespace() + { + return "default"; + } +} diff --git a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java new file mode 100644 index 000000000000..b892ac8165b9 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java @@ -0,0 +1,118 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.metamx.http.client.HttpClient; + +import javax.servlet.DispatcherType; +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import java.io.IOException; +import java.util.EnumSet; +import java.util.Map; + +public class NoopAuthenticator implements Authenticator +{ + @Override + public Class getFilterClass() + { + return null; + } + + @Override + public Map getInitParameters() + { + return null; + } + + @Override + public String getPath() + { + return "/*"; + } + + @Override + public EnumSet getDispatcherType() + { + return null; + } + + @Override + public String getTypeName() + { + return "noop"; + } + + @Override + public String getNamespace() + { + return "noop"; + } + + @Override + public Filter getFilter() + { + return new Filter() + { + @Override + public void init(FilterConfig filterConfig) throws ServletException + { + + } + + @Override + public void doFilter( + ServletRequest request, ServletResponse response, FilterChain chain + ) throws IOException, ServletException + { + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN, "druid"); + chain.doFilter(request, response); + } + + @Override + public void destroy() + { + + } + }; + } + + @Override + public String getAuthChallengeHeader() + { + return null; + } + + @Override + public boolean authenticateJDBCContext(Map context) + { + return true; + } + + @Override + public HttpClient createInternalClient(HttpClient baseClient) + { + return baseClient; + } +} diff --git a/server/src/main/java/io/druid/server/security/SystemAuthorizationInfo.java b/server/src/main/java/io/druid/server/security/NoopAuthorizationManager.java similarity index 69% rename from server/src/main/java/io/druid/server/security/SystemAuthorizationInfo.java rename to server/src/main/java/io/druid/server/security/NoopAuthorizationManager.java index f41983edc696..e2a93527ba7a 100644 --- a/server/src/main/java/io/druid/server/security/SystemAuthorizationInfo.java +++ b/server/src/main/java/io/druid/server/security/NoopAuthorizationManager.java @@ -19,21 +19,17 @@ package io.druid.server.security; -/** - * An AuthorizationInfo that is useful for actions generated internally by the system. It allows everything. - */ -public class SystemAuthorizationInfo implements AuthorizationInfo +public class NoopAuthorizationManager implements AuthorizationManager { - public static final SystemAuthorizationInfo INSTANCE = new SystemAuthorizationInfo(); - - private SystemAuthorizationInfo() + @Override + public Access authorize(String identity, Resource resource, Action action) { - // Singleton. + return new Access(true); } @Override - public Access isAuthorized(final Resource resource, final Action action) + public String getNamespace() { - return new Access(true); + return "noop"; } } diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java new file mode 100644 index 000000000000..045eec6a7a37 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -0,0 +1,265 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.QueryInterruptedException; +import io.druid.server.DruidNode; +import org.eclipse.jetty.server.Response; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletOutputStream; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.WriteListener; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.servlet.http.HttpServletResponseWrapper; +import java.io.BufferedWriter; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; + +/** + * Filter that verifies that authorization checks were applied to an HTTP request, before sending a response. + *

+ * This filter is intended to help catch missing authorization checks arising from bugs/design omissions. + */ +public class PreResponseAuthorizationCheckFilter implements Filter +{ + private static final Logger log = new Logger(PreResponseAuthorizationCheckFilter.class); + + private final AuthConfig authConfig; + private final List authenticators; + private final ObjectMapper jsonMapper; + + public PreResponseAuthorizationCheckFilter( + AuthConfig authConfig, + List authenticators, + ObjectMapper jsonMapper + ) + { + this.authConfig = authConfig; + this.authenticators = authenticators; + this.jsonMapper = jsonMapper; + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException + { + + } + + @Override + public void doFilter( + ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain + ) throws IOException, ServletException + { + if (authConfig.isEnabled()) { + QueryInterruptedException unauthorizedError = new QueryInterruptedException( + QueryInterruptedException.UNAUTHORIZED, + null, + null, + DruidNode.getDefaultHost() + ); + unauthorizedError.setStackTrace(new StackTraceElement[0]); + OutputStream out = servletResponse.getOutputStream(); + + Boolean authInfoChecked = null; + final HttpServletResponse response = (HttpServletResponse) servletResponse; + + // Since this is the last filter in the chain, some previous authentication filter + // should have placed an auth token in the request. + // If not, send an auth challenge. + if (servletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) == null) { + Set supportedAuthSchemes = Sets.newHashSet(); + for (Authenticator authenticator : authenticators) { + String challengeHeader = authenticator.getAuthChallengeHeader(); + if (challengeHeader != null) { + supportedAuthSchemes.add(challengeHeader); + } + } + for (String authScheme : supportedAuthSchemes) { + response.addHeader("WWW-Authenticate", authScheme); + } + sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); + return; + } + + // capture the response stream before its sent to client, or we don't get a chance to modify it later + // http://www.oracle.com/technetwork/java/filters-137243.html + GenericResponseWrapper wrapper = new GenericResponseWrapper((HttpServletResponse) servletResponse); + filterChain.doFilter(servletRequest, wrapper); + + // After response has been generated, something in the request processing path must have set + // DRUID_AUTH_TOKEN_CHECKED (i.e. performed an authorization check). If this is not set, + // a 403 error will be returned instead of the response. + authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); + if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { + log.error( + "Request did not have an authorization check performed: %s", + ((HttpServletRequest) servletRequest).getRequestURI() + ); + sendJsonError(response, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); + } else { + out.write(wrapper.getData()); + } + out.close(); + } else { + filterChain.doFilter(servletRequest, servletResponse); + } + } + + @Override + public void destroy() + { + + } + + private static boolean errorOverridesMissingAuth(int status) + { + return status == Response.SC_INTERNAL_SERVER_ERROR; + } + + public static void sendJsonError(HttpServletResponse resp, int error, String errorJson, OutputStream outputStream) + { + resp.setStatus(error); + resp.setContentType("application/json"); + resp.setCharacterEncoding("UTF-8"); + try { + outputStream.write(errorJson.getBytes(StandardCharsets.UTF_8)); + } + catch (IOException ioe) { + log.error("WTF? Can't get writer from HTTP response."); + } + } + + // classes from "Servlet Filters and Event Listeners" + // https://docs.oracle.com/cd/B14099_19/web.1012/b14017/filters.htm + private static class GenericResponseWrapper extends HttpServletResponseWrapper + { + private ByteArrayOutputStream output; + private int contentLength; + private String contentType; + + public GenericResponseWrapper(HttpServletResponse response) + { + super(response); + output = new ByteArrayOutputStream(); + } + + public byte[] getData() + { + return output.toByteArray(); + } + + @Override + public ServletOutputStream getOutputStream() + { + return new FilterServletOutputStream(output); + } + + @Override + public PrintWriter getWriter() + { + return new PrintWriter( + new BufferedWriter(new OutputStreamWriter(getOutputStream(), StandardCharsets.UTF_8)), + true + ); + } + + @Override + public void setContentLength(int length) + { + this.contentLength = length; + super.setContentLength(length); + } + + public int getContentLength() + { + return contentLength; + } + + @Override + public void setContentType(String type) + { + this.contentType = type; + super.setContentType(type); + } + + @Override + public String getContentType() + { + return contentType; + } + } + + private static class FilterServletOutputStream extends ServletOutputStream + { + + private DataOutputStream stream; + + public FilterServletOutputStream(OutputStream output) + { + stream = new DataOutputStream(output); + } + + @Override + public void write(int b) throws IOException + { + stream.write(b); + } + + @Override + public void write(byte[] b) throws IOException + { + stream.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + stream.write(b, off, len); + } + + @Override + public boolean isReady() + { + return false; + } + + @Override + public void setWriteListener(WriteListener writeListener) + { + + } + } +} diff --git a/server/src/main/java/io/druid/server/security/Resource.java b/server/src/main/java/io/druid/server/security/Resource.java index d3c74fb52899..e2b1e86292b7 100644 --- a/server/src/main/java/io/druid/server/security/Resource.java +++ b/server/src/main/java/io/druid/server/security/Resource.java @@ -19,22 +19,31 @@ package io.druid.server.security; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + public class Resource { private final String name; private final ResourceType type; - public Resource(String name, ResourceType type) + @JsonCreator + public Resource( + @JsonProperty("name") String name, + @JsonProperty("type") ResourceType type + ) { this.name = name; this.type = type; } + @JsonProperty public String getName() { return name; } + @JsonProperty public ResourceType getType() { return type; diff --git a/server/src/main/java/io/druid/server/security/ResourceAction.java b/server/src/main/java/io/druid/server/security/ResourceAction.java new file mode 100644 index 000000000000..5c70729cd4ae --- /dev/null +++ b/server/src/main/java/io/druid/server/security/ResourceAction.java @@ -0,0 +1,51 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class ResourceAction +{ + private final Resource resource; + private final Action action; + + @JsonCreator + public ResourceAction( + @JsonProperty("resource") Resource resource, + @JsonProperty("action") Action action + ) + { + this.resource = resource; + this.action = action; + } + + @JsonProperty + public Resource getResource() + { + return resource; + } + + @JsonProperty + public Action getAction() + { + return action; + } +} diff --git a/server/src/main/java/io/druid/server/security/ResourceType.java b/server/src/main/java/io/druid/server/security/ResourceType.java index 818bf9ca947d..e3bc1dfadfe0 100644 --- a/server/src/main/java/io/druid/server/security/ResourceType.java +++ b/server/src/main/java/io/druid/server/security/ResourceType.java @@ -19,9 +19,20 @@ package io.druid.server.security; +import com.fasterxml.jackson.annotation.JsonCreator; + public enum ResourceType { DATASOURCE, CONFIG, - STATE + STATE; + + @JsonCreator + public static ResourceType fromString(String name) + { + if (name == null) { + return null; + } + return valueOf(name.toUpperCase()); + } } diff --git a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java new file mode 100644 index 000000000000..51867dd0a603 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java @@ -0,0 +1,102 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.java.util.common.logger.Logger; +import io.druid.query.QueryInterruptedException; +import io.druid.server.DruidNode; +import org.eclipse.jetty.server.Response; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; + +public class SecuritySanityCheckFilter implements Filter +{ + private static final Logger log = new Logger(SecuritySanityCheckFilter.class); + + private final ObjectMapper jsonMapper; + + public SecuritySanityCheckFilter( + ObjectMapper jsonMapper + ) + { + this.jsonMapper = jsonMapper; + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException + { + + } + + @Override + public void doFilter( + ServletRequest request, ServletResponse response, FilterChain chain + ) throws IOException, ServletException + { + HttpServletResponse httpResponse = (HttpServletResponse) response; + QueryInterruptedException unauthorizedError = new QueryInterruptedException( + QueryInterruptedException.UNAUTHORIZED, + null, + null, + DruidNode.getDefaultHost() + ); + unauthorizedError.setStackTrace(new StackTraceElement[0]); + OutputStream out = httpResponse.getOutputStream(); + + // make sure the original request isn't trying to fake the auth token checks + Boolean authInfoChecked = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); + Boolean authToken = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); + if (authInfoChecked != null || authToken != null) { + sendJsonError(httpResponse, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); + return; + } + + chain.doFilter(request, response); + } + + @Override + public void destroy() + { + + } + + public static void sendJsonError(HttpServletResponse resp, int error, String errorJson, OutputStream outputStream) + { + resp.setStatus(error); + resp.setContentType("application/json"); + resp.setCharacterEncoding("UTF-8"); + try { + outputStream.write(errorJson.getBytes(StandardCharsets.UTF_8)); + } + catch (IOException ioe) { + log.error("WTF? Can't get writer from HTTP response."); + } + } +} diff --git a/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java b/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java new file mode 100644 index 000000000000..712c63b583e8 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java @@ -0,0 +1,56 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import java.io.IOException; + +public class UnsecuredResourceFilter implements Filter +{ + public UnsecuredResourceFilter() + { + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException + { + + } + + @Override + public void doFilter( + ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain + ) throws IOException, ServletException + { + servletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + filterChain.doFilter(servletRequest, servletResponse); + } + + @Override + public void destroy() + { + + } +} diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index b16c6684f458..3b1a4251f227 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -42,6 +42,7 @@ import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; +import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -339,7 +340,8 @@ public CallbackAction segmentViewInitialized() baseView, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), new NoopServiceEmitter(), - new BrokerSegmentWatcherConfig() + new BrokerSegmentWatcherConfig(), + new AuthConfig() ); baseView.start(); diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index 60b5ffd28b82..6a5c00704fa0 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -52,7 +52,10 @@ import io.druid.server.log.RequestLogger; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.router.QueryHostFinder; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.NoopAuthorizationManager; import org.eclipse.jetty.client.HttpClient; +import io.druid.server.security.AuthorizationManager; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -111,6 +114,16 @@ public void configure(Binder binder) binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null, null, new ServerConfig()) ); binder.bind(JettyServerInitializer.class).to(ProxyJettyServerInit.class).in(LazySingleton.class); + binder.bind(AuthorizationManagerMapper.class).toInstance( + new AuthorizationManagerMapper(null) { + + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new NoopAuthorizationManager(); + } + } + ); Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); diff --git a/server/src/test/java/io/druid/server/ClientInfoResourceTest.java b/server/src/test/java/io/druid/server/ClientInfoResourceTest.java index ae0988a25865..94e88b066b33 100644 --- a/server/src/test/java/io/druid/server/ClientInfoResourceTest.java +++ b/server/src/test/java/io/druid/server/ClientInfoResourceTest.java @@ -414,7 +414,13 @@ private ClientInfoResource getResourceTestHelper( SegmentMetadataQueryConfig segmentMetadataQueryConfig ) { - return new ClientInfoResource(serverInventoryView, timelineServerView, segmentMetadataQueryConfig, new AuthConfig()) + return new ClientInfoResource( + serverInventoryView, + timelineServerView, + segmentMetadataQueryConfig, + new AuthConfig(), + null + ) { @Override protected DateTime getCurrentTime() diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index 1dc226b979f8..fda855ce75f5 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -45,7 +45,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.server.security.Resource; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -113,6 +114,7 @@ public QueryRunner getQueryRunnerForSegments( } }; + private static final ServiceEmitter noopServiceEmitter = new NoopServiceEmitter(); private QueryResource queryResource; @@ -139,12 +141,15 @@ public void setup() new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig() + new AuthConfig(), + new AuthorizationManagerMapper(null) ), jsonMapper, jsonMapper, queryManager, - new AuthConfig() + new AuthConfig(), + null, + new DefaultGenericQueryMetricsFactory(jsonMapper) ); } @@ -166,16 +171,9 @@ public void setup() @Test public void testGoodQuery() throws IOException { - EasyMock.expect(testServletRequest.getAttribute(EasyMock.anyString())).andReturn( - new AuthorizationInfo() - { - @Override - public Access isAuthorized(Resource resource, Action action) - { - return new Access(true); - } - } - ).times(1); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + EasyMock.replay(testServletRequest); Response response = queryResource.doPost( new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")), @@ -201,13 +199,26 @@ public void testBadQuery() throws IOException @Test public void testSecuredQuery() throws Exception { - EasyMock.expect(testServletRequest.getAttribute(EasyMock.anyString())).andReturn( - new AuthorizationInfo() + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + + testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + EasyMock.expectLastCall().times(1); + + testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + EasyMock.expectLastCall().times(1); + + EasyMock.replay(testServletRequest); + + AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new AuthorizationManager() { @Override - public Access isAuthorized( - Resource resource, Action action - ) + public Access authorize(String identity, Resource resource, Action action) { if (resource.getName().equals("allow")) { return new Access(true); @@ -215,9 +226,15 @@ public Access isAuthorized( return new Access(false); } } - } - ).times(2); - EasyMock.replay(testServletRequest); + + @Override + public String getNamespace() + { + return null; + } + }; + } + }; queryResource = new QueryResource( new QueryLifecycleFactory( @@ -227,12 +244,15 @@ public Access isAuthorized( new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper ), jsonMapper, jsonMapper, queryManager, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper, + new DefaultGenericQueryMetricsFactory(jsonMapper) ); Response response = queryResource.doPost( @@ -260,13 +280,23 @@ public void testSecuredGetServer() throws Exception final CountDownLatch startAwaitLatch = new CountDownLatch(1); final CountDownLatch cancelledCountDownLatch = new CountDownLatch(1); - EasyMock.expect(testServletRequest.getAttribute(EasyMock.anyString())).andReturn( - new AuthorizationInfo() + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + + testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + EasyMock.expectLastCall().times(1); + + EasyMock.replay(testServletRequest); + + AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new AuthorizationManager() { @Override - public Access isAuthorized( - Resource resource, Action action - ) + public Access authorize(String identity, Resource resource, Action action) { // READ action corresponds to the query // WRITE corresponds to cancellation of query @@ -289,9 +319,15 @@ public Access isAuthorized( return new Access(true); } } - } - ).times(2); - EasyMock.replay(testServletRequest); + + @Override + public String getNamespace() + { + return null; + } + }; + } + }; queryResource = new QueryResource( new QueryLifecycleFactory( @@ -301,12 +337,15 @@ public Access isAuthorized( new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper ), jsonMapper, jsonMapper, queryManager, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper, + new DefaultGenericQueryMetricsFactory(jsonMapper) ); final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"," @@ -366,13 +405,27 @@ public void testDenySecuredGetServer() throws Exception final CountDownLatch waitFinishLatch = new CountDownLatch(2); final CountDownLatch startAwaitLatch = new CountDownLatch(1); - EasyMock.expect(testServletRequest.getAttribute(EasyMock.anyString())).andReturn( - new AuthorizationInfo() + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + + testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + EasyMock.expectLastCall().times(1); + + testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + EasyMock.expectLastCall().times(1); + + EasyMock.replay(testServletRequest); + + AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) + { + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new AuthorizationManager() { @Override - public Access isAuthorized( - Resource resource, Action action - ) + public Access authorize(String identity, Resource resource, Action action) { // READ action corresponds to the query // WRITE corresponds to cancellation of query @@ -389,9 +442,15 @@ public Access isAuthorized( return new Access(false); } } - } - ).times(2); - EasyMock.replay(testServletRequest); + + @Override + public String getNamespace() + { + return null; + } + }; + } + }; queryResource = new QueryResource( new QueryLifecycleFactory( @@ -401,12 +460,15 @@ public Access isAuthorized( new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper ), jsonMapper, jsonMapper, queryManager, - new AuthConfig(true) + new AuthConfig(true, null, null, null), + authMapper, + new DefaultGenericQueryMetricsFactory(jsonMapper) ); final String queryString = "{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"," diff --git a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java index b9e31b486037..13bd0f3e7830 100644 --- a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java +++ b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java @@ -29,7 +29,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.server.security.Resource; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; @@ -121,7 +122,7 @@ public void testGetFullQueryableDataSources() throws Exception ImmutableList.of(server) ).atLeastOnce(); EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); DruidDataSource[] resultantDruidDataSources = new DruidDataSource[result.size()]; @@ -142,19 +143,27 @@ public void testGetFullQueryableDataSources() throws Exception @Test public void testSecuredGetFullQueryableDataSources() throws Exception { + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); EasyMock.expect(server.getDataSources()).andReturn( ImmutableList.of(listDataSources.get(0), listDataSources.get(1)) ).atLeastOnce(); + + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); - EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn( - new AuthorizationInfo() + EasyMock.replay(inventoryView, server, request); + + AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new AuthorizationManager() { @Override - public Access isAuthorized( - Resource resource, Action action - ) + public Access authorize(String identity, Resource resource, Action action) { if (resource.getName().equals("datasource1")) { return new Access(true); @@ -162,11 +171,23 @@ public Access isAuthorized( return new Access(false); } } - } - ).times(2); - EasyMock.replay(inventoryView, server, request); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(true)); + @Override + public String getNamespace() + { + return null; + } + }; + } + }; + + DatasourcesResource datasourcesResource = new DatasourcesResource( + inventoryView, + null, + null, + new AuthConfig(true, null, null, null), + authMapper + ); Response response = datasourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); DruidDataSource[] resultantDruidDataSources = new DruidDataSource[result.size()]; @@ -205,7 +226,7 @@ public void testGetSimpleQueryableDataSources() throws Exception ).atLeastOnce(); EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getQueryableDataSources(null, "simple", request); Assert.assertEquals(200, response.getStatus()); List> results = (List>) response.getEntity(); @@ -232,7 +253,7 @@ public void testFullGetTheDataSource() throws Exception ).atLeastOnce(); EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getTheDataSource("datasource1", "full"); DruidDataSource result = (DruidDataSource) response.getEntity(); Assert.assertEquals(200, response.getStatus()); @@ -249,7 +270,7 @@ public void testNullGetTheDataSource() throws Exception ).atLeastOnce(); EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Assert.assertEquals(204, datasourcesResource.getTheDataSource("none", null).getStatus()); EasyMock.verify(inventoryView, server); } @@ -271,7 +292,7 @@ public void testSimpleGetTheDataSource() throws Exception ).atLeastOnce(); EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getTheDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); @@ -310,7 +331,7 @@ public void testSimpleGetTheDataSourceManyTiers() throws Exception ).atLeastOnce(); EasyMock.replay(inventoryView, server, server2, server3); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getTheDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); @@ -341,7 +362,7 @@ public void testGetSegmentDataSourceIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(new Interval("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getSegmentDataSourceIntervals("invalidDataSource", null, null); Assert.assertEquals(response.getEntity(), null); @@ -388,7 +409,7 @@ public void testGetSegmentDataSourceSpecificInterval() ).atLeastOnce(); EasyMock.replay(inventoryView); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getSegmentDataSourceSpecificInterval( "invalidDataSource", "2010-01-01/P1D", @@ -455,7 +476,13 @@ public void testDeleteDataSourceSpecificInterval() throws Exception EasyMock.expectLastCall().once(); EasyMock.replay(indexingServiceClient, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, indexingServiceClient, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource( + inventoryView, + null, + indexingServiceClient, + new AuthConfig(), + null + ); Response response = datasourcesResource.deleteDataSourceSpecificInterval("datasource1", interval); Assert.assertEquals(200, response.getStatus()); @@ -468,7 +495,13 @@ public void testDeleteDataSource() { IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); EasyMock.replay(indexingServiceClient, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, indexingServiceClient, new AuthConfig()); + DatasourcesResource datasourcesResource = new DatasourcesResource( + inventoryView, + null, + indexingServiceClient, + new AuthConfig(), + null + ); Response response = datasourcesResource.deleteDataSource("datasource", "true", "???"); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); diff --git a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java index 3c437cc8f2a0..220a176e9b36 100644 --- a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java +++ b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java @@ -110,7 +110,7 @@ public void testGetIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); expectedIntervals.add(new Interval("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); Response response = intervalsResource.getIntervals(request); TreeMap>> actualIntervals = (TreeMap) response.getEntity(); @@ -136,7 +136,7 @@ public void testSimpleGetSpecificIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", "simple", null, request); Map> actualIntervals = (Map) response.getEntity(); @@ -157,7 +157,7 @@ public void testFullGetSpecificIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, "full", request); TreeMap>> actualIntervals = (TreeMap) response.getEntity(); @@ -178,14 +178,13 @@ public void testGetSpecificIntervals() ).atLeastOnce(); EasyMock.replay(inventoryView); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, null, request); Map actualIntervals = (Map) response.getEntity(); Assert.assertEquals(2, actualIntervals.size()); Assert.assertEquals(25L, actualIntervals.get("size")); Assert.assertEquals(2, actualIntervals.get("count")); - } @After diff --git a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java index c594a14bd2bf..3d70f18cda24 100644 --- a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java +++ b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java @@ -36,7 +36,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationInfo; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.server.security.Resource; import org.easymock.EasyMock; @@ -56,18 +57,19 @@ public class ResourceFilterTestHelper { public HttpServletRequest req; - public AuthorizationInfo authorizationInfo; + public AuthorizationManagerMapper authorizationManagerMapper; public ContainerRequest request; public void setUp(ResourceFilter resourceFilter) throws Exception { req = EasyMock.createStrictMock(HttpServletRequest.class); request = EasyMock.createStrictMock(ContainerRequest.class); - authorizationInfo = EasyMock.createStrictMock(AuthorizationInfo.class); + authorizationManagerMapper = EasyMock.createStrictMock(AuthorizationManagerMapper.class); // Memory barrier synchronized (this) { ((AbstractResourceFilter) resourceFilter).setReq(req); + ((AbstractResourceFilter) resourceFilter).setAuthorizationManagerMapper(authorizationManagerMapper); } } @@ -107,14 +109,29 @@ public MultivaluedMap getMatrixParameters() ) ).anyTimes(); EasyMock.expect(request.getMethod()).andReturn(requestMethod).anyTimes(); - EasyMock.expect(req.getAttribute(EasyMock.anyString())).andReturn(authorizationInfo).atLeastOnce(); - EasyMock.expect(authorizationInfo.isAuthorized( - EasyMock.anyObject(Resource.class), - EasyMock.anyObject(Action.class) + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").atLeastOnce(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("namespace").atLeastOnce(); + req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, authCheckResult); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(authorizationManagerMapper.getAuthorizationManager( + EasyMock.anyString() )).andReturn( - new Access(authCheckResult) - ).atLeastOnce(); + new AuthorizationManager() + { + @Override + public Access authorize(String identity, Resource resource, Action action) + { + return new Access(authCheckResult); + } + @Override + public String getNamespace() + { + return null; + } + } + ).atLeastOnce(); } public static Collection getRequestPaths(final Class clazz) @@ -122,6 +139,11 @@ public static Collection getRequestPaths(final Class clazz) return getRequestPaths(clazz, ImmutableList.>of(), ImmutableList.>of()); } + public static Collection getRequestPathsWithAuthorizationManager(final Class clazz) + { + return getRequestPaths(clazz, ImmutableList.>of(AuthorizationManagerMapper.class), ImmutableList.>of()); + } + public static Collection getRequestPaths( final Class clazz, final Iterable> mockableInjections @@ -162,7 +184,7 @@ public void configure(Binder binder) for (Key key : mockableKeys) { binder.bind((Key) key).toInstance(EasyMock.createNiceMock(key.getTypeLiteral().getRawType())); } - binder.bind(AuthConfig.class).toInstance(new AuthConfig(true)); + binder.bind(AuthConfig.class).toInstance(new AuthConfig(true, null, null, null)); } } ); diff --git a/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java index 31803052e427..8c62576a4ddb 100644 --- a/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java @@ -56,20 +56,20 @@ public static Collection data() { return ImmutableList.copyOf( Iterables.concat( - getRequestPaths(CoordinatorResource.class), - getRequestPaths(DatasourcesResource.class), - getRequestPaths(BrokerResource.class), - getRequestPaths(HistoricalResource.class), - getRequestPaths(IntervalsResource.class), - getRequestPaths(MetadataResource.class), - getRequestPaths(RulesResource.class), - getRequestPaths(ServersResource.class), - getRequestPaths(TiersResource.class), - getRequestPaths(ClientInfoResource.class), - getRequestPaths(CoordinatorDynamicConfigsResource.class), - getRequestPaths(QueryResource.class), - getRequestPaths(StatusResource.class), - getRequestPaths(BrokerQueryResource.class) + getRequestPathsWithAuthorizationManager(CoordinatorResource.class), + getRequestPathsWithAuthorizationManager(DatasourcesResource.class), + getRequestPathsWithAuthorizationManager(BrokerResource.class), + getRequestPathsWithAuthorizationManager(HistoricalResource.class), + getRequestPathsWithAuthorizationManager(IntervalsResource.class), + getRequestPathsWithAuthorizationManager(MetadataResource.class), + getRequestPathsWithAuthorizationManager(RulesResource.class), + getRequestPathsWithAuthorizationManager(ServersResource.class), + getRequestPathsWithAuthorizationManager(TiersResource.class), + getRequestPathsWithAuthorizationManager(ClientInfoResource.class), + getRequestPathsWithAuthorizationManager(CoordinatorDynamicConfigsResource.class), + getRequestPathsWithAuthorizationManager(QueryResource.class), + getRequestPathsWithAuthorizationManager(StatusResource.class), + getRequestPathsWithAuthorizationManager(BrokerQueryResource.class) ) ); } @@ -102,17 +102,17 @@ public void setUp() throws Exception public void testResourcesFilteringAccess() { setUpMockExpectations(requestPath, true, requestMethod); - EasyMock.replay(req, request, authorizationInfo); + EasyMock.replay(req, request, authorizationManagerMapper); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); resourceFilter.getRequestFilter().filter(request); - EasyMock.verify(req, request, authorizationInfo); + EasyMock.verify(req, request, authorizationManagerMapper); } @Test(expected = WebApplicationException.class) public void testResourcesFilteringNoAccess() { setUpMockExpectations(requestPath, false, requestMethod); - EasyMock.replay(req, request, authorizationInfo); + EasyMock.replay(req, request, authorizationManagerMapper); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); try { resourceFilter.getRequestFilter().filter(request); @@ -121,16 +121,16 @@ public void testResourcesFilteringNoAccess() Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), e.getResponse().getStatus()); throw e; } - EasyMock.verify(req, request, authorizationInfo); + EasyMock.verify(req, request, authorizationManagerMapper); } @Test public void testResourcesFilteringBadPath() { - EasyMock.replay(req, request, authorizationInfo); + EasyMock.replay(req, request, authorizationManagerMapper); final String badRequestPath = requestPath.replaceAll("\\w+", "droid"); Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath)); - EasyMock.verify(req, request, authorizationInfo); + EasyMock.verify(req, request, authorizationManagerMapper); } } diff --git a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java index 8897c7f553f2..91d893115cd6 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java @@ -43,6 +43,9 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.jetty.JettyBindings; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.NoopAuthorizationManager; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -75,6 +78,16 @@ public void configure(Binder binder) Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); + binder.bind(AuthorizationManagerMapper.class).toInstance( + new AuthorizationManagerMapper(null) { + + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new NoopAuthorizationManager(); + } + } + ); JettyBindings.addQosFilter(binder, "/slow/*", 2); final ServerConfig serverConfig = new ObjectMapper().convertValue( ImmutableMap.of("numThreads", "10"), diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index a38c1340f536..74234527b498 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -41,6 +41,9 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.initialization.jetty.ServletFilterHolder; +import io.druid.server.security.AuthorizationManager; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.NoopAuthorizationManager; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Server; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -126,6 +129,16 @@ public EnumSet getDispatcherType() Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); + binder.bind(AuthorizationManagerMapper.class).toInstance( + new AuthorizationManagerMapper(null) { + + @Override + public AuthorizationManager getAuthorizationManager(String namespace) + { + return new NoopAuthorizationManager(); + } + } + ); LifecycleModule.register(binder, Server.class); } } diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 9ae4b5233309..91a57607f024 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -19,7 +19,9 @@ package io.druid.cli; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; @@ -44,6 +46,7 @@ import io.druid.guice.ListProvider; import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; +import io.druid.guice.annotations.Json; import io.druid.indexing.common.actions.LocalTaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionToolbox; @@ -84,6 +87,9 @@ import io.druid.server.http.RedirectInfo; import io.druid.server.initialization.jetty.JettyServerInitUtils; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import io.druid.tasklogs.TaskLogStreamer; import io.druid.tasklogs.TaskLogs; import org.eclipse.jetty.server.Handler; @@ -107,6 +113,14 @@ public class CliOverlord extends ServerRunnable { private static Logger log = new Logger(CliOverlord.class); + private static List UNSECURED_PATHS = Lists.newArrayList( + "/", + "/console.html", + "/old-console/*", + "/images/*", + "/js/*" + ); + public CliOverlord() { super(log); @@ -289,8 +303,35 @@ public void initialize(Server server, Injector injector) } ) ); + + final AuthConfig authConfig = injector.getInstance(AuthConfig.class); + final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + List authenticators = null; + if (authConfig.isEnabled()) { + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + + } + JettyServerInitUtils.addExtensionFilters(root, injector); + if (authConfig.isEnabled()) { + // perform no-op authorization for these static resources + AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); + + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); + } + // /status should not redirect, so add first root.addFilter(GuiceFilter.class, "/status/*", null); diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 953ef3b4b6f6..9676cb41c387 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -139,7 +139,7 @@ protected List getModules() public void configure(Binder binder) { binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/peon"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(-1); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); PolyBind.createChoice( diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 4ec581918be2..7b35dcd77cab 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -19,14 +19,22 @@ package io.druid.cli; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; import com.google.inject.Inject; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.servlet.GuiceFilter; +import io.druid.guice.annotations.Json; +import io.druid.java.util.common.logger.Logger; import io.druid.server.coordinator.DruidCoordinatorConfig; import io.druid.server.http.OverlordProxyServlet; import io.druid.server.http.RedirectFilter; import io.druid.server.initialization.jetty.JettyServerInitUtils; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -37,12 +45,29 @@ import org.eclipse.jetty.util.resource.Resource; import org.eclipse.jetty.util.resource.ResourceCollection; +import java.util.List; import java.util.Properties; /** */ class CoordinatorJettyServerInitializer implements JettyServerInitializer { + private static List UNSECURED_PATHS = Lists.newArrayList( + "/", + "/favicon.ico", + "/css/*", + "/druid.js", + "/druid.css", + "/pages/*", + "/druid/*", + "/fonts/*", + "/old-console/*", + "/coordinator/*", + "/overlord/*" + ); + + private static Logger log = new Logger(CoordinatorJettyServerInitializer.class); + private final DruidCoordinatorConfig config; private final boolean beOverlord; @@ -81,8 +106,34 @@ public void initialize(Server server, Injector injector) // used for console development root.setResourceBase(config.getConsoleStatic()); } + + final AuthConfig authConfig = injector.getInstance(AuthConfig.class); + final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + List authenticators = null; + if (authConfig.isEnabled()) { + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + } + JettyServerInitUtils.addExtensionFilters(root, injector); + if (authConfig.isEnabled()) { + // perform no-op authorization for these static resources + AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); + + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); + } + // /status should not redirect, so add first root.addFilter(GuiceFilter.class, "/status/*", null); diff --git a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java index 48c99baa6edc..5fdb776ff5e7 100644 --- a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java @@ -19,10 +19,17 @@ package io.druid.cli; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.servlet.GuiceFilter; +import io.druid.guice.annotations.Json; +import io.druid.java.util.common.logger.Logger; import io.druid.server.initialization.jetty.JettyServerInitUtils; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -31,16 +38,44 @@ import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import java.util.List; + /** */ class MiddleManagerJettyServerInitializer implements JettyServerInitializer { + private static Logger log = new Logger(MiddleManagerJettyServerInitializer.class); + @Override public void initialize(Server server, Injector injector) { final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + + final AuthConfig authConfig = injector.getInstance(AuthConfig.class); + final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + List authenticators = null; + if (authConfig.isEnabled()) { + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + } + JettyServerInitUtils.addExtensionFilters(root, injector); + + if (authConfig.isEnabled()) { + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); + } + root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java index d3d3f8b9fd43..ba6f7f7925a6 100644 --- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -19,12 +19,19 @@ package io.druid.cli; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.inject.Inject; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.servlet.GuiceFilter; +import io.druid.guice.annotations.Json; +import io.druid.java.util.common.logger.Logger; import io.druid.server.initialization.jetty.JettyServerInitUtils; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -39,6 +46,8 @@ */ public class QueryJettyServerInitializer implements JettyServerInitializer { + private static Logger log = new Logger(QueryJettyServerInitializer.class); + private final List extensionHandlers; @Inject @@ -52,8 +61,31 @@ public void initialize(Server server, Injector injector) { final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + + final AuthConfig authConfig = injector.getInstance(AuthConfig.class); + final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + List authenticators = null; + if (authConfig.isEnabled()) { + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + } + JettyServerInitUtils.addExtensionFilters(root, injector); + if (authConfig.isEnabled()) { + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); + } + root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 7369fc4e0d88..044d38422f64 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -19,13 +19,20 @@ package io.druid.cli; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.servlet.GuiceFilter; +import io.druid.guice.annotations.Json; import io.druid.guice.http.DruidHttpClientConfig; +import io.druid.java.util.common.logger.Logger; import io.druid.server.AsyncQueryForwardingServlet; import io.druid.server.initialization.jetty.JettyServerInitUtils; import io.druid.server.initialization.jetty.JettyServerInitializer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -33,10 +40,14 @@ import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import java.util.List; + /** */ public class RouterJettyServerInitializer implements JettyServerInitializer { + private static Logger log = new Logger(RouterJettyServerInitializer.class); + private final AsyncQueryForwardingServlet asyncQueryForwardingServlet; private final DruidHttpClientConfig httpClientConfig; @@ -63,7 +74,31 @@ public void initialize(Server server, Injector injector) sh.setInitParameter("maxThreads", Integer.toString(httpClientConfig.getNumMaxThreads())); root.addServlet(sh, "/druid/v2/*"); + + final AuthConfig authConfig = injector.getInstance(AuthConfig.class); + final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + List authenticators = null; + if (authConfig.isEnabled()) { + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + } + JettyServerInitUtils.addExtensionFilters(root, injector); + + if (authConfig.isEnabled()) { + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); + } + // Can't use '/*' here because of Guice conflicts with AsyncQueryForwardingServlet path root.addFilter(GuiceFilter.class, "/status/*", null); root.addFilter(GuiceFilter.class, "/druid/router/*", null); diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java b/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java index e6779458b57c..471efce6a904 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidAvaticaHandler.java @@ -56,9 +56,6 @@ public void handle( final HttpServletResponse response ) throws IOException, ServletException { - // This is not integrated with the experimental authorization framework. - // (Non-trivial since we don't know the dataSources up-front) - if (request.getRequestURI().equals(AVATICA_PATH)) { super.handle(target, baseRequest, request, response); } diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java index 8e78800f22fa..2eb29ac7ccf6 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java @@ -138,4 +138,9 @@ public DruidConnection sync(final Future newTimeoutFuture) } return this; } + + public Map context() + { + return context; + } } diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java index 222c063f654f..e2eb27db9215 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java @@ -29,9 +29,13 @@ import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; +import com.google.inject.Injector; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.Authenticator; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerFactory; import org.apache.calcite.avatica.MetaImpl; @@ -63,6 +67,8 @@ public class DruidMeta extends MetaImpl private final PlannerFactory plannerFactory; private final ScheduledExecutorService exec; private final AvaticaServerConfig config; + private final AuthConfig authConfig; + private final List authenticators; // Used to track logical connections. private final Map connections = new ConcurrentHashMap<>(); @@ -72,17 +78,32 @@ public class DruidMeta extends MetaImpl private final AtomicInteger connectionCount = new AtomicInteger(); @Inject - public DruidMeta(final PlannerFactory plannerFactory, final AvaticaServerConfig config) + public DruidMeta( + final PlannerFactory plannerFactory, + final AvaticaServerConfig config, + final AuthConfig authConfig, + final Injector injector + ) { super(null); this.plannerFactory = Preconditions.checkNotNull(plannerFactory, "plannerFactory"); this.config = config; + this.authConfig = authConfig; this.exec = Executors.newSingleThreadScheduledExecutor( new ThreadFactoryBuilder() .setNameFormat(StringUtils.format("DruidMeta@%s-ScheduledExecutor", Integer.toHexString(hashCode()))) .setDaemon(true) .build() ); + + if (authConfig.isEnabled()) { + this.authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + } else { + this.authenticators = null; + } } @Override @@ -91,9 +112,9 @@ public void openConnection(final ConnectionHandle ch, final Map // Build connection context. final ImmutableMap.Builder context = ImmutableMap.builder(); for (Map.Entry entry : info.entrySet()) { - if (!SKIP_PROPERTIES.contains(entry.getKey())) { + //if (!SKIP_PROPERTIES.contains(entry.getKey())) { context.put(entry); - } + //} } openDruidConnection(ch.id, context.build()); } @@ -132,7 +153,17 @@ public StatementHandle prepare( { final StatementHandle statement = createStatement(ch); final DruidStatement druidStatement = getDruidStatement(statement); - statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount).getSignature(); + final DruidConnection druidConnection = getDruidConnection(statement.connectionId); + String user = null; + String namespace = null; + if (authConfig.isEnabled()) { + if (!authenticateConnection(druidConnection)) { + throw new SecurityException("Authentication failed."); + } + user = getConnectionUser(druidConnection); + namespace = getConnectionNamespace(druidConnection); + } + statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, user, namespace).getSignature(); return statement; } @@ -160,7 +191,18 @@ public ExecuteResult prepareAndExecute( { // Ignore "callback", this class is designed for use with LocalService which doesn't use it. final DruidStatement druidStatement = getDruidStatement(statement); - final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount).getSignature(); + final DruidConnection druidConnection = getDruidConnection(statement.connectionId); + String user = null; + String namespace = null; + if (authConfig != null && authConfig.isEnabled()) { + if (!authenticateConnection(druidConnection)) { + throw new SecurityException("Authentication failed."); + } + ; + user = getConnectionUser(druidConnection); + namespace = getConnectionNamespace(druidConnection); + } + final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, user, namespace).getSignature(); final Frame firstFrame = druidStatement.execute() .nextFrame( DruidStatement.START_OFFSET, @@ -487,6 +529,30 @@ void closeAllConnections() } } + private boolean authenticateConnection(final DruidConnection connection) + { + Map context = connection.context(); + for (Authenticator authenticator : authenticators) { + if (authenticator.authenticateJDBCContext(context)) { + context.put("namespace", authenticator.getNamespace()); + return true; + } + } + return false; + } + + private String getConnectionUser(final DruidConnection connection) + { + Map context = connection.context(); + return (String) context.get("user"); + } + + private String getConnectionNamespace(final DruidConnection connection) + { + Map context = connection.context(); + return (String) context.get("namespace"); + } + private DruidConnection openDruidConnection(final String connectionId, final Map context) { if (connectionCount.incrementAndGet() > config.getMaxConnections()) { diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java index 07e8e26f5881..47604c083b60 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java @@ -152,12 +152,18 @@ public static List createColumnMetaData(final RelDataType rowTyp return columns; } - public DruidStatement prepare(final PlannerFactory plannerFactory, final String query, final long maxRowCount) + public DruidStatement prepare( + final PlannerFactory plannerFactory, + final String query, + final long maxRowCount, + final String user, + final String namespace + ) { try (final DruidPlanner planner = plannerFactory.createPlanner(queryContext)) { synchronized (lock) { ensure(State.NEW); - this.plannerResult = planner.plan(query); + this.plannerResult = planner.plan(query, null, user, namespace); this.maxRowCount = maxRowCount; this.query = query; this.signature = Meta.Signature.create( diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java index 773e4b67f6bf..6f628cde8ab0 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java @@ -23,8 +23,13 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.server.security.Access; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizationUtils; import io.druid.sql.calcite.rel.DruidConvention; import io.druid.sql.calcite.rel.DruidRel; import org.apache.calcite.DataContext; @@ -34,9 +39,11 @@ import org.apache.calcite.interpreter.Bindables; import org.apache.calcite.linq4j.Enumerable; import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.RelRoot; +import org.apache.calcite.rel.RelVisitor; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; @@ -50,22 +57,43 @@ import org.apache.calcite.tools.ValidationException; import org.apache.calcite.util.Pair; +import javax.servlet.http.HttpServletRequest; import java.io.Closeable; import java.util.ArrayList; import java.util.List; +import java.util.Set; public class DruidPlanner implements Closeable { private final Planner planner; private final PlannerContext plannerContext; + private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; - public DruidPlanner(final Planner planner, final PlannerContext plannerContext) + public DruidPlanner( + final Planner planner, + final PlannerContext plannerContext, + final AuthConfig authConfig, + final AuthorizationManagerMapper authorizationManagerMapper + ) { this.planner = planner; this.plannerContext = plannerContext; + this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } public PlannerResult plan(final String sql) throws SqlParseException, ValidationException, RelConversionException + { + return plan(sql, null, null, null); + } + + public PlannerResult plan( + final String sql, + final HttpServletRequest request, + final String user, + final String namespace + ) throws SqlParseException, ValidationException, RelConversionException, SecurityException { SqlExplain explain = null; SqlNode parsed = planner.parse(sql); @@ -77,12 +105,12 @@ public PlannerResult plan(final String sql) throws SqlParseException, Validation final RelRoot root = planner.rel(validated); try { - return planWithDruidConvention(explain, root); + return planWithDruidConvention(explain, root, request, user, namespace); } catch (RelOptPlanner.CannotPlanException e) { // Try again with BINDABLE convention. Used for querying Values, metadata tables, and fallback. try { - return planWithBindableConvention(explain, root); + return planWithBindableConvention(explain, root, request, user, namespace); } catch (Exception e2) { e.addSuppressed(e2); @@ -104,8 +132,11 @@ public void close() private PlannerResult planWithDruidConvention( final SqlExplain explain, - final RelRoot root - ) throws RelConversionException + final RelRoot root, + final HttpServletRequest request, + final String user, + final String namespace + ) throws RelConversionException, SecurityException { final DruidRel druidRel = (DruidRel) planner.transform( Rules.DRUID_CONVENTION_RULES, @@ -115,6 +146,31 @@ private PlannerResult planWithDruidConvention( root.rel ); + if (authConfig != null && authConfig.isEnabled()) { + List datasourceNames = druidRel.getDatasourceNames(); + Access authResult; + if (request != null) { + authResult = AuthorizationUtils.authorizeAllResourceActions( + request, + datasourceNames, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizationManagerMapper + ); + } else { + authResult = AuthorizationUtils.authorizeAllResourceActions( + datasourceNames, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + user, + namespace, + authorizationManagerMapper + ); + } + + if (!authResult.isAllowed()) { + throw new SecurityException(authResult.toString()); + } + } + if (explain != null) { return planExplanation(druidRel, explain); } else { @@ -149,9 +205,52 @@ public Object[] apply(final Object[] input) } } + private Access authorizeBindableRel(BindableRel rel, HttpServletRequest req, final String user, final String namespace) + { + Set datasourceNames = Sets.newHashSet(); + rel.childrenAccept( + new RelVisitor() + { + @Override + public void visit(RelNode node, int ordinal, RelNode parent) + { + if (node instanceof DruidRel) { + datasourceNames.addAll(((DruidRel) node).getDatasourceNames()); + } + if (node instanceof Bindables.BindableTableScan) { + Bindables.BindableTableScan bts = (Bindables.BindableTableScan) node; + RelOptTable table = bts.getTable(); + String tableName = table.getQualifiedName().get(0); + datasourceNames.add(tableName); + } + node.childrenAccept(this); + } + } + ); + if (req != null) { + return AuthorizationUtils.authorizeAllResourceActions( + req, + datasourceNames, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizationManagerMapper + ); + } else { + return AuthorizationUtils.authorizeAllResourceActions( + datasourceNames, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + user, + namespace, + authorizationManagerMapper + ); + } + } + private PlannerResult planWithBindableConvention( final SqlExplain explain, - final RelRoot root + final RelRoot root, + final HttpServletRequest request, + final String user, + final String namespace ) throws RelConversionException { BindableRel bindableRel = (BindableRel) planner.transform( @@ -177,6 +276,13 @@ private PlannerResult planWithBindableConvention( root.validatedRowType ); } + + if (authConfig != null && authConfig.isEnabled()) { + Access accessResult = authorizeBindableRel(bindableRel, request, user, namespace); + if (!accessResult.isAllowed()) { + throw new SecurityException(accessResult.toString()); + } + } if (explain != null) { return planExplanation(bindableRel, explain); diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java index d75adc5bfb6a..57b1e852815e 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java @@ -22,6 +22,8 @@ import com.google.inject.Inject; import io.druid.math.expr.ExprMacroTable; import io.druid.server.QueryLifecycleFactory; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.schema.DruidSchema; import org.apache.calcite.avatica.util.Casing; @@ -54,13 +56,18 @@ public class PlannerFactory private final ExprMacroTable macroTable; private final PlannerConfig plannerConfig; + private final AuthConfig authConfig; + private final AuthorizationManagerMapper authorizationManagerMapper; + @Inject public PlannerFactory( final DruidSchema druidSchema, final QueryLifecycleFactory queryLifecycleFactory, final DruidOperatorTable operatorTable, final ExprMacroTable macroTable, - final PlannerConfig plannerConfig + final PlannerConfig plannerConfig, + final AuthConfig authConfig, + final AuthorizationManagerMapper authorizationManagerMapper ) { this.druidSchema = druidSchema; @@ -68,6 +75,8 @@ public PlannerFactory( this.operatorTable = operatorTable; this.macroTable = macroTable; this.plannerConfig = plannerConfig; + this.authConfig = authConfig; + this.authorizationManagerMapper = authorizationManagerMapper; } public DruidPlanner createPlanner(final Map queryContext) @@ -89,6 +98,6 @@ public DruidPlanner createPlanner(final Map queryContext) .typeSystem(DruidTypeSystem.INSTANCE) .build(); - return new DruidPlanner(Frameworks.getPlanner(frameworkConfig), plannerContext); + return new DruidPlanner(Frameworks.getPlanner(frameworkConfig), plannerContext, authConfig, authorizationManagerMapper); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java index 3ada2785f16a..acc832e55608 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidNestedGroupBy.java @@ -155,6 +155,12 @@ public DruidNestedGroupBy asDruidConvention() ); } + @Override + public List getDatasourceNames() + { + return sourceRel.getDatasourceNames(); + } + @Override public RelWriter explainTerms(RelWriter pw) { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java index 64d91a1e5498..595fed6ec3f4 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryRel.java @@ -40,6 +40,8 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.calcite.rel.type.RelDataType; +import java.util.List; + public class DruidQueryRel extends DruidRel { // Factors used for computing cost (see computeSelfCost). These are intended to encourage pushing down filters @@ -130,6 +132,12 @@ public DruidQueryRel asDruidConvention() ); } + @Override + public List getDatasourceNames() + { + return druidTable.getDataSource().getNames(); + } + @Override public RowSignature getSourceRowSignature() { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java index d1071bc57f27..c091c9c80716 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidRel.java @@ -35,6 +35,8 @@ import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.AbstractRelNode; +import java.util.List; + public abstract class DruidRel extends AbstractRelNode implements BindableRel { private final QueryMaker queryMaker; @@ -90,6 +92,11 @@ public PlannerContext getPlannerContext() public abstract T asDruidConvention(); + /** + * Get a list of names of datasources read by this DruidRel + */ + public abstract List getDatasourceNames(); + @Override public Class getElementType() { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java index 65fafeeee9a1..f2536fa9dce1 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java @@ -179,6 +179,16 @@ public DruidSemiJoin asDruidConvention() ); } + @Override + public List getDatasourceNames() + { + Set datasourceNames = Sets.newLinkedHashSet(); + datasourceNames.addAll(left.getDatasourceNames()); + datasourceNames.addAll(right.getDatasourceNames()); + + return Lists.newArrayList(datasourceNames); + } + @Override public int getQueryCount() { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 2faf3db2bd8c..f653c2e1c69f 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -230,9 +230,8 @@ private Sequence runQuery(final Query query) { Hook.QUERY_PLAN.run(query); - // Authorization really should be applied in planning. At this point the query has already begun to execute. - // So, use "null" authorizationInfo to force the query to fail if security is enabled. - return queryLifecycleFactory.factorize().runSimple(query, null, null); + // Authorization has already been checked during planning, skip authorization checks here. + return queryLifecycleFactory.factorize().runSimple(query, null, null, null, false); } private Sequence executeTimeseries( diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index 41e07dbb47b5..cf017ce1db5b 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -49,7 +49,6 @@ import io.druid.segment.column.ValueType; import io.druid.server.QueryLifecycleFactory; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.security.SystemAuthorizationInfo; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; import io.druid.sql.calcite.table.RowSignature; @@ -495,8 +494,8 @@ private static Sequence runSegmentMetadataQuery( false ); - // Use SystemAuthorizationInfo since this is a query generated by Druid itself. - return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, SystemAuthorizationInfo.INSTANCE, null); + // This is an internally generated query, no authorization is needed. + return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, null, null, null, false); } private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis) diff --git a/sql/src/main/java/io/druid/sql/http/SqlResource.java b/sql/src/main/java/io/druid/sql/http/SqlResource.java index 9c8cf6bdff14..c5d361df409b 100644 --- a/sql/src/main/java/io/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/io/druid/sql/http/SqlResource.java @@ -40,11 +40,13 @@ import org.joda.time.DateTimeZone; import org.joda.time.format.ISODateTimeFormat; +import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.Produces; import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; @@ -74,16 +76,16 @@ public SqlResource( @POST @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) - public Response doPost(final SqlQuery sqlQuery) throws SQLException, IOException + public Response doPost( + final SqlQuery sqlQuery, + @Context final HttpServletRequest req + ) throws SQLException, IOException { - // This is not integrated with the experimental authorization framework. - // (Non-trivial since we don't know the dataSources up-front) - final PlannerResult plannerResult; final DateTimeZone timeZone; try (final DruidPlanner planner = plannerFactory.createPlanner(sqlQuery.getContext())) { - plannerResult = planner.plan(sqlQuery.getQuery()); + plannerResult = planner.plan(sqlQuery.getQuery(), req, null, null); timeZone = planner.getPlannerContext().getTimeZone(); // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values. diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index bd96ce110574..d4e4a3e65bda 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -35,6 +35,8 @@ import io.druid.math.expr.ExprMacroTable; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -125,13 +127,17 @@ public void setUp() throws Exception CalciteTests.createMockQueryLifecycleFactory(walker), operatorTable, macroTable, - plannerConfig + plannerConfig, + new AuthConfig(), + new AuthorizationManagerMapper(null) ), - AVATICA_CONFIG + AVATICA_CONFIG, + new AuthConfig(), + null ); final DruidAvaticaHandler handler = new DruidAvaticaHandler( druidMeta, - new DruidNode("dummy", "dummy", 1, null, new ServerConfig()), + new DruidNode("dummy", "dummy", 1, null, new ServerConfig()), new AvaticaMonitor() ); final int port = new Random().nextInt(9999) + 10000; @@ -143,7 +149,7 @@ public void setUp() throws Exception port, DruidAvaticaHandler.AVATICA_PATH ); - client = DriverManager.getConnection(url); + client = DriverManager.getConnection(url, "admin", "druid"); final Properties propertiesLosAngeles = new Properties(); propertiesLosAngeles.setProperty("sqlTimeZone", "America/Los_Angeles"); @@ -564,9 +570,13 @@ public int getMaxRowsPerFrame() CalciteTests.createMockQueryLifecycleFactory(walker), operatorTable, macroTable, - plannerConfig + plannerConfig, + new AuthConfig(), + new AuthorizationManagerMapper(null) ), - smallFrameConfig + smallFrameConfig, + new AuthConfig(), + null ) { @Override diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java index af10663a8a3b..eb325800644c 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; import io.druid.math.expr.ExprMacroTable; +import io.druid.server.security.AuthConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -70,7 +71,9 @@ public void setUp() throws Exception CalciteTests.createMockQueryLifecycleFactory(walker), operatorTable, macroTable, - plannerConfig + plannerConfig, + new AuthConfig(), + null ); } @@ -85,7 +88,7 @@ public void tearDown() throws Exception public void testSignature() throws Exception { final String sql = "SELECT * FROM druid.foo"; - final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1); + final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1, null, null); // Check signature. final Meta.Signature signature = statement.getSignature(); @@ -124,7 +127,7 @@ public List apply(final ColumnMetaData columnMetaData) public void testSelectAllInFirstFrame() throws Exception { final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo"; - final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1); + final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1, null, null); // First frame, ask for all rows. Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 6); @@ -150,8 +153,8 @@ public void testSelectAllInFirstFrame() throws Exception public void testSelectSplitOverTwoFrames() throws Exception { final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo"; - final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1); - + final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1, null, null); + // First frame, ask for 2 rows. Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 2); Assert.assertEquals( diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 156ed170bb34..9794cacd0612 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -82,6 +82,7 @@ import io.druid.segment.column.Column; import io.druid.segment.column.ValueType; import io.druid.segment.virtual.ExpressionVirtualColumn; +import io.druid.server.security.AuthConfig; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -5444,7 +5445,9 @@ private List getResults( CalciteTests.createMockQueryLifecycleFactory(walker), operatorTable, macroTable, - plannerConfig + plannerConfig, + new AuthConfig(), + null ); viewManager.createView( diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java index ab75a343d84d..ade19aa5720b 100644 --- a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java @@ -29,6 +29,7 @@ import io.druid.math.expr.ExprMacroTable; import io.druid.query.QueryInterruptedException; import io.druid.query.ResourceLimitExceededException; +import io.druid.server.security.AuthConfig; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -84,7 +85,9 @@ public void setUp() throws Exception CalciteTests.createMockQueryLifecycleFactory(walker), operatorTable, macroTable, - plannerConfig + plannerConfig, + new AuthConfig(), + null ) ); } @@ -237,7 +240,7 @@ public void testResourceLimitExceeded() throws Exception // Returns either an error or a result. private Pair>> doPost(final SqlQuery query) throws Exception { - final Response response = resource.doPost(query); + final Response response = resource.doPost(query, null); if (response.getStatus() == 200) { final StreamingOutput output = (StreamingOutput) response.getEntity(); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index fc3f30e0c0a2..109de6959a00 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -88,6 +88,7 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.log.NoopRequestLogger; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthorizationManagerMapper; import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.SqlOperatorConversion; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -311,7 +312,8 @@ public > QueryToolChest getToolChest new ServiceEmitter("dummy", "dummy", new NoopEmitter()), new NoopRequestLogger(), new ServerConfig(), - new AuthConfig() + new AuthConfig(), + new AuthorizationManagerMapper(null) ); } From 037f5dd5d12fe04dcb71f8f4a5da127d761bfb26 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Tue, 25 Jul 2017 16:51:31 -0700 Subject: [PATCH 02/34] Address some PR comments --- docs/content/configuration/auth.md | 39 +++++---- .../io/druid/client/BrokerServerView.java | 6 +- .../java/io/druid/server/QueryLifecycle.java | 6 +- .../java/io/druid/server/QueryResource.java | 2 +- .../http/security/SecurityResourceFilter.java | 85 ------------------- .../io/druid/server/security/AuthConfig.java | 27 +++--- .../druid/server/security/Authenticator.java | 27 ++++-- .../server/security/AuthorizationManager.java | 11 +++ .../server/security/AuthorizationUtils.java | 49 ++--------- .../io/druid/client/BrokerServerViewTest.java | 4 +- .../java/io/druid/sql/avatica/DruidMeta.java | 6 +- 11 files changed, 82 insertions(+), 180 deletions(-) delete mode 100644 server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index f1972037a7a0..87d08b5782e2 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -5,44 +5,51 @@ layout: doc_page # Authentication and Authorization |Property|Type|Description|Default|Required| -|--------|-----------|-------| +|--------|-----------|--------|--------|--------| |`druid.auth.enabled`|boolean|Determines if authentication and authorization checks will be performed on requests.|false|no| -|`druid.auth.authenticationChainPath`|String|Path to a file containing a JSON list of Authenticator objects|null|yes| -|`druid.auth.internalAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes| -|`druid.auth.authorizationManager`|String|Type of the AuthorizationManager to be used for authorization checks.|"default"|no| +|`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|null|yes, if auth enabled| +|`druid.auth.internalAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes, if auth enabled| +|`druid.auth.authorizationManagers`|JSON List of Strings|List of AuthorizationManager type names |null|yes, if auth enabled| ## Enabling Authentication/Authorization ## Authentication Chain -Authentication decisions are handled by a chain of Authenticator instances. A request will be checked by Authenticators in the sequence defined by the `druid.auth.authenticationChainPath` file. +Authentication decisions are handled by a chain of Authenticator instances. A request will be checked by Authenticators in the sequence defined by the `druid.auth.authenticationChain` file. Authenticator implementions are provided by extensions. For example, the following authentication chain definition enables the Kerberos and HTTP Basic authenticators, from the `druid-kerberos` and `druid-basic-security` core extensions, respectively: ```json -[ -{ - "type": "kerberos" -}, -{ - "type": "basic" -} -] +["kerberos", "basic"] ``` ## Internal Authenticator The `druid.auth.internalAuthenticator` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a broker node communicates with historical nodes for query processing). -The Authenticator chosen for this property must also be present in the Authentication Chain. +The Authenticator chosen for this property must also be present in `druid.auth.authenticationChain`. -## Authorization Manager -Authorization decisions are handled by an AuthorizationManager. The `druid.auth.authorizationManager` property determines what type of AuthorizationManager will be used. +## Authorization Managers +Authorization decisions are handled by an AuthorizationManager. The `druid.auth.authorizationManagers` property determines what AuthorizationManager implementations will be active. There are two built-in AuthorizationManagers, "default" and "noop". Other implementations are provided by extensions. +For example, the following authorization managers definition enables the "basic" implementation from `druid-basic-security`: + +```json +["basic"] +``` + ### Default Authorization Manager The default AuthorizationManager with type name "default" rejects all requests. ### No-op Authorization Manager The no-op AuthorizationManager with type name "noop" accepts all requests. + + +## Namespaces +Authenticator and AuthorizationManager implementations are linked through a namespace string. Authenticators tag an authenticated request with a namespace, which is used to route the authenticated request to the AuthorizationManager implementation that registered itself with a matching namespace. + +This is to support cases where an AuthorizationManager implementation is only intended to authorize requests from a specific authenticator (an implementation may have assumptions about the user name format, for example). + +The details of namespace configuration are left for implementors of Authenticator and AuthorizationManager to decide. \ No newline at end of file diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index ea0b68ed2004..13495c552654 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -40,7 +40,6 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -75,7 +74,6 @@ public class BrokerServerView implements TimelineServerView private final TierSelectorStrategy tierSelectorStrategy; private final ServiceEmitter emitter; private final Predicate> segmentFilter; - private final AuthConfig authConfig; private volatile boolean initialized = false; @@ -88,8 +86,7 @@ public BrokerServerView( FilteredServerInventoryView baseView, TierSelectorStrategy tierSelectorStrategy, ServiceEmitter emitter, - final BrokerSegmentWatcherConfig segmentWatcherConfig, - AuthConfig authConfig + final BrokerSegmentWatcherConfig segmentWatcherConfig ) { this.warehouse = warehouse; @@ -102,7 +99,6 @@ public BrokerServerView( this.clients = Maps.newConcurrentMap(); this.selectors = Maps.newHashMap(); this.timelines = Maps.newHashMap(); - this.authConfig = authConfig; this.segmentFilter = new Predicate>() { diff --git a/server/src/main/java/io/druid/server/QueryLifecycle.java b/server/src/main/java/io/druid/server/QueryLifecycle.java index c913f5970a3d..510f61c616ae 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycle.java +++ b/server/src/main/java/io/druid/server/QueryLifecycle.java @@ -57,7 +57,7 @@ * *
    *
  1. Initialization ({@link #initialize(Query)})
  2. - *
  3. Authorization ({@link #authorize(AuthorizationInfo)}
  4. + *
  5. Authorization ({@link #authorize(String, String, HttpServletRequest)}
  6. *
  7. Execution ({@link #execute()}
  8. *
  9. Logging ({@link #emitLogsAndMetrics(Throwable, String, long)}
  10. *
@@ -114,8 +114,8 @@ public QueryLifecycle( * either fully iterated or throws an exception. * * @param query the query - * @param authorizationInfo authorization info from the request; or null if none is present. This must be non-null - * if security is enabled, or the request will be considered unauthorized. + * @param user authentication token from the request + * @param namespace authentication namespace of the request * @param remoteAddress remote address, for logging; or null if unknown * @param needsAuth if false, skip the authorization check. This is useful when the authorization check has * already been performed (e.g. in SQL handling, where authorization takes place in the diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index b133661b3c87..ebcbaaf24d51 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -156,7 +156,7 @@ public Response getServer(@PathParam("id") String queryId, @Context final HttpSe public Response doPost( final InputStream in, @QueryParam("pretty") final String pretty, - @Context final HttpServletRequest req // used to get request content-type, remote address and AuthorizationInfo + @Context final HttpServletRequest req // used to get request content-type, remote address and auth-related headers ) throws IOException { final QueryLifecycle queryLifecycle = queryLifecycleFactory.factorize(); diff --git a/server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java b/server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java deleted file mode 100644 index 65d06fe31d5b..000000000000 --- a/server/src/main/java/io/druid/server/http/security/SecurityResourceFilter.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.server.http.security; - -import com.google.common.collect.ImmutableList; -import com.google.inject.Inject; -import com.sun.jersey.spi.container.ContainerRequest; -import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; -import io.druid.server.security.AuthorizationUtils; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceAction; -import io.druid.server.security.ResourceType; - -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; -import java.util.List; - -public class SecurityResourceFilter extends AbstractResourceFilter -{ - @Inject - public SecurityResourceFilter( - AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper - ) - { - super(authConfig, authorizationManagerMapper); - } - - @Override - public ContainerRequest filter(ContainerRequest request) - { - if (getAuthConfig().isEnabled()) { - final ResourceAction resourceAction = new ResourceAction( - new Resource("security", ResourceType.CONFIG), - getAction(request) - ); - - final Access authResult = AuthorizationUtils.authorizeResourceAction( - getReq(), - resourceAction, - getAuthorizationManagerMapper() - ); - - if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(String.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); - } - } - return request; - } - - @Override - public boolean isApplicable(String requestPath) - { - List applicablePaths = ImmutableList.of("druid/coordinator/v1/security/"); - for (String path : applicablePaths) { - if(requestPath.startsWith(path) && !requestPath.equals(path)) { - return true; - } - } - return false; - } -} diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index f9ebf95bc550..06875338d67d 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -27,7 +27,7 @@ public class AuthConfig { /** - * Use this String as the attribute name for the request attribute to pass {@link AuthorizationInfo} + * Use this String as the attribute name for the request attribute to pass an authentication token * from the servlet filter to the jersey resource */ public static final String DRUID_AUTH_TOKEN = "Druid-Auth-Token"; @@ -63,11 +63,6 @@ public AuthConfig( this.authorizationManagers = authorizationManagers; } - /** - * If druid.auth.enabled is set to true then an implementation of AuthorizationInfo - * must be provided and it must be set as a request attribute possibly inside the servlet filter - * injected in the filter chain using your own extension - */ @JsonProperty private final boolean enabled; @@ -107,6 +102,7 @@ public String toString() "enabled=" + enabled + ", authenticatorChain='" + authenticatorChain + '\'' + ", internalAuthenticator='" + internalAuthenticator + '\'' + + ", authorizationManagers='" + authorizationManagers + '\'' + '}'; } @@ -125,10 +121,19 @@ public boolean equals(Object o) if (isEnabled() != that.isEnabled()) { return false; } - if (!getInternalAuthenticator().equals(that.getInternalAuthenticator())) { + if (getAuthenticatorChain() != null + ? !getAuthenticatorChain().equals(that.getAuthenticatorChain()) + : that.getAuthenticatorChain() != null) { return false; } - return getAuthenticatorChain().equals(that.getAuthenticatorChain()); + if (getInternalAuthenticator() != null + ? !getInternalAuthenticator().equals(that.getInternalAuthenticator()) + : that.getInternalAuthenticator() != null) { + return false; + } + return getAuthorizationManagers() != null + ? getAuthorizationManagers().equals(that.getAuthorizationManagers()) + : that.getAuthorizationManagers() == null; } @@ -136,8 +141,10 @@ public boolean equals(Object o) public int hashCode() { int result = (isEnabled() ? 1 : 0); - result = 31 * result + getAuthenticatorChain().hashCode(); - result = 31 * result + getInternalAuthenticator().hashCode(); + result = 31 * result + (getAuthenticatorChain() != null ? getAuthenticatorChain().hashCode() : 0); + result = 31 * result + (getInternalAuthenticator() != null ? getInternalAuthenticator().hashCode() : 0); + result = 31 * result + (getAuthorizationManagers() != null ? getAuthorizationManagers().hashCode() : 0); return result; } + } diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index d27c351ddbb3..2302e930c8ec 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -31,6 +31,15 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "noop", value = NoopAuthenticator.class), }) +/** + * This interface is essentially a ServletFilterHolder with additional requirements on the getFilter() method contract, plus: + * + * - A method that returns a WWW-Authenticate challenge header appropriate for the authentication mechanism. + * - A method for creating a wrapped HTTP client that can authenticate using the Authenticator's authentication scheme, + * used for internal Druid node communications (e.g., broker -> historical messages) + * - A method for authenticating credentials contained in a JDBC connection context, used for authenticating Druid SQL + * requests received via JDBC + */ public interface Authenticator extends ServletFilterHolder { /** @@ -46,16 +55,16 @@ public interface Authenticator extends ServletFilterHolder /** * Create a Filter that performs authentication checks on incoming HTTP requests. - * + *

* If the authentication succeeds, the Filter should set the "Druid-Auth-Token" attribute in the request, * containing a String that represents the authenticated identity of the requester. - * + *

* If the "Druid-Auth-Token" attribute is already set (i.e., request has been authenticated by an earlier Filter), * this Filter should skip any authentication checks and proceed to the next Filter. - * + *

* If the authentication fails, the Filter should not send an error response. The error response will be sent * after all Filters in the authentication filter chain have been checked. - * + *

* If an anonymous request is received, the Filter should continue on to the next Filter, the challenge response * will be sent after the filter chain is exhausted. * @@ -66,7 +75,7 @@ public interface Authenticator extends ServletFilterHolder /** * Return a WWW-Authenticate challenge scheme string appropriate for this Authenticator's authentication mechanism. - * + *

* For example, a Basic HTTP implementation should return "Basic", while a Kerberos implementation would return * "Negotiate". * @@ -77,12 +86,13 @@ public interface Authenticator extends ServletFilterHolder /** * Given a JDBC connection context, authenticate the identity represented by the information in the context. * This is used to secure JDBC access for Druid SQL. - * + *

* For example, a Basic HTTP auth implementation could read the "user" and "password" fields from the JDBC context. - * + *

* The expected contents of the context are left to the implementation. * * @param context JDBC connection context + * * @return true if the identity represented by the context is successfully authenticated */ public boolean authenticateJDBCContext(Map context); @@ -90,11 +100,12 @@ public interface Authenticator extends ServletFilterHolder /** * Return a client that sends requests with the format/information necessary to authenticate successfully * against this Authenticator's authentication scheme using the identity of the internal system user. - * + *

* This HTTP client is used for internal communications between Druid nodes, such as when a broker communicates * with a historical node during query processing. * * @param baseClient Base HTTP client for internal Druid communications + * * @return HttpClient that sends requests with the credentials of the internal system user */ public HttpClient createInternalClient(HttpClient baseClient); diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManager.java b/server/src/main/java/io/druid/server/security/AuthorizationManager.java index ae5be80debfb..da1c902cf2ff 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationManager.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationManager.java @@ -27,6 +27,17 @@ @JsonSubTypes.Type(name = "default", value = DefaultAuthorizationManager.class), @JsonSubTypes.Type(name = "noop", value = NoopAuthorizationManager.class) }) +/** + * An AuthorizationManager is responsible for performing authorization checks for resource accesses. + * + * A single instance of each AuthorizationManager implementation will be created per node. + * Security-sensitive endpoints will need to extract the identity string contained in the request's Druid-Auth-Token + * attribute, previously set by an Authenticator. Each endpoint will pass this identity String to the + * AuthorizationManager's authorize() method along with any Resource/Action pairs created for the request being + * handled. The endpoint can use these checks to filter out resources or deny the request as needed. + * After a request is authorized, a new attribute, "Druid-Auth-Token-Checked", should be set in the + * request header with the result of the authorization decision. + */ public interface AuthorizationManager { /** diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index 6091195ea83b..f827853d1348 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -36,7 +36,7 @@ public class AuthorizationUtils public final static Access ACCESS_OK = new Access(true, "All resource-actions authorized."); /** - * Check a resource-action using the AuthorizationInfo from the request. + * Check a resource-action using the authorization fields from the request. * * Otherwise, if the resource-actions is authorized, return ACCESS_OK. * @@ -46,7 +46,8 @@ public class AuthorizationUtils * * @param request HTTP request to be authorized * @param resourceAction A resource identifier and the action to be taken the resource. - * @return ACCESS_OK or the failed Access object returned by the request's AuthorizationInfo. + * @param authorizationManagerMapper The singleton AuthorizationManagerMapper instance + * @return ACCESS_OK or the failed Access object returned by the AuthorizationManager that checked the request. */ public static Access authorizeResourceAction( final HttpServletRequest request, @@ -63,7 +64,7 @@ public static Access authorizeResourceAction( /** - * Check a list of resource-actions using the AuthorizationInfo from the request. + * Check a list of resource-actions using the authorization fields from the request. * * If one of the resource-actions fails the authorization check, this method returns the failed * Access object from the check. @@ -219,47 +220,7 @@ public static Access authorizeAllResourceActions( } /** - * Check a list of resource-actions using the AuthorizationInfo from the request. - * - * If one of the resource-actions fails the authorization check, this method returns the failed - * Access object from the check. - * - * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. - * - * @param resourceActions A list of resource-actions to authorize - * @return ACCESS_OK or the Access object from the first failed check - */ - public static Access authorizeAllResourceActions( - final String user, - final String namespace, - final AuthorizationManagerMapper authorizationManagerMapper, - final List resourceActions - ) - { - if (user == null || namespace == null) { - throw new ISE("null user or namespace"); - } - final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); - if (authorizationManager == null) { - throw new ISE("No authorization manager found for namespace: [%s].", namespace); - } - - for (ResourceAction resourceAction : resourceActions) { - final Access access = authorizationManager.authorize( - user, - resourceAction.getResource(), - resourceAction.getAction() - ); - if (!access.isAllowed()) { - return access; - } - } - - return ACCESS_OK; - } - - /** - * Filter a list of resource-actions using the request's AuthorizationInfo, returning a new list of + * Filter a list of resource-actions using the request's authorization fields, returning a new list of * resource-actions that were authorized. * * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index 3b1a4251f227..b16c6684f458 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -42,7 +42,6 @@ import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -340,8 +339,7 @@ public CallbackAction segmentViewInitialized() baseView, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), new NoopServiceEmitter(), - new BrokerSegmentWatcherConfig(), - new AuthConfig() + new BrokerSegmentWatcherConfig() ); baseView.start(); diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java index e2eb27db9215..ee0050e6d9da 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java @@ -52,7 +52,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -62,7 +61,6 @@ public class DruidMeta extends MetaImpl { private static final Logger log = new Logger(DruidMeta.class); - private static final Set SKIP_PROPERTIES = ImmutableSet.of("user", "password"); private final PlannerFactory plannerFactory; private final ScheduledExecutorService exec; @@ -112,9 +110,7 @@ public void openConnection(final ConnectionHandle ch, final Map // Build connection context. final ImmutableMap.Builder context = ImmutableMap.builder(); for (Map.Entry entry : info.entrySet()) { - //if (!SKIP_PROPERTIES.contains(entry.getKey())) { - context.put(entry); - //} + context.put(entry); } openDruidConnection(ch.id, context.build()); } From 6dd0af9b234f4acecbf3def7035f8f86afe9fb3d Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 26 Jul 2017 16:18:03 -0700 Subject: [PATCH 03/34] Authorization result caching --- .../server/security/AuthorizationUtils.java | 41 +++++++++++++++---- .../druid/server/security/ResourceAction.java | 27 ++++++++++++ 2 files changed, 60 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index f827853d1348..915efd877a7d 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -21,12 +21,14 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import io.druid.java.util.common.ISE; import javax.servlet.http.HttpServletRequest; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; /** * Static utility functions for performing authorization checks. @@ -62,7 +64,6 @@ public static Access authorizeResourceAction( ); } - /** * Check a list of resource-actions using the authorization fields from the request. * @@ -99,7 +100,12 @@ public static Access authorizeAllResourceActions( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } + final Map resultCache = Maps.newHashMap(); for (ResourceAction resourceAction : resourceActions) { + // this method returns on first failure, so only successful Access results are kept in the cache + if (resultCache.get(resourceAction) != null) { + continue; + } final Access access = authorizationManager.authorize( identity, resourceAction.getResource(), @@ -108,6 +114,8 @@ public static Access authorizeAllResourceActions( if (!access.isAllowed()) { request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); return access; + } else { + resultCache.put(resourceAction, access); } } @@ -115,7 +123,6 @@ public static Access authorizeAllResourceActions( return ACCESS_OK; } - /** * Check a list of caller-defined resources, after converting them into a list of resource-actions * using a caller provided function. @@ -157,8 +164,13 @@ public static Access authorizeAllResourceActions( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } + final Map resultCache = Maps.newHashMap(); for (ResType resource : resources) { final ResourceAction resourceAction = raGenerator.apply(resource); + // this method returns on first failure, so only successful Access results are kept in the cache + if (resultCache.get(resourceAction) != null) { + continue; + } final Access access = authorizationManager.authorize( identity, resourceAction.getResource(), @@ -167,6 +179,8 @@ public static Access authorizeAllResourceActions( if (!access.isAllowed()) { request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); return access; + } else { + resultCache.put(resourceAction, access); } } @@ -204,8 +218,13 @@ public static Access authorizeAllResourceActions( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } + final Map resultCache = Maps.newHashMap(); for (ResType resource : resources) { final ResourceAction resourceAction = raGenerator.apply(resource); + // this method returns on first failure, so only successful Access results are kept in the cache + if (resultCache.get(resourceAction) != null) { + continue; + } final Access access = authorizationManager.authorize( user, resourceAction.getResource(), @@ -213,6 +232,8 @@ public static Access authorizeAllResourceActions( ); if (!access.isAllowed()) { return access; + } else { + resultCache.put(resourceAction, access); } } @@ -232,7 +253,6 @@ public static Access authorizeAllResourceActions( * @param resourceActionGenerator Function that creates a resource-action from a resource * @return A list containing the resource-actions from the resourceParser that were successfully authorized. */ - public static List filterAuthorizedResources( final HttpServletRequest request, final Collection resources, @@ -255,14 +275,19 @@ public static List filterAuthorizedResources( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } + final Map resultCache = Maps.newHashMap(); List filteredResources = new ArrayList<>(); for (ResType resource : resources) { final ResourceAction resourceAction = resourceActionGenerator.apply(resource); - final Access access = authorizationManager.authorize( - identity, - resourceAction.getResource(), - resourceAction.getAction() - ); + Access access = resultCache.get(resourceAction); + if (access == null) { + access = authorizationManager.authorize( + identity, + resourceAction.getResource(), + resourceAction.getAction() + ); + resultCache.put(resourceAction, access); + } if (access.isAllowed()) { filteredResources.add(resource); } diff --git a/server/src/main/java/io/druid/server/security/ResourceAction.java b/server/src/main/java/io/druid/server/security/ResourceAction.java index 5c70729cd4ae..240f9280562f 100644 --- a/server/src/main/java/io/druid/server/security/ResourceAction.java +++ b/server/src/main/java/io/druid/server/security/ResourceAction.java @@ -48,4 +48,31 @@ public Action getAction() { return action; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ResourceAction that = (ResourceAction) o; + + if (!getResource().equals(that.getResource())) { + return false; + } + return getAction() == that.getAction(); + + } + + @Override + public int hashCode() + { + int result = getResource().hashCode(); + result = 31 * result + getAction().hashCode(); + return result; + } } From bcb3074b7d01b00ed061ad7bde76a4e93398e005 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 26 Jul 2017 18:27:50 -0700 Subject: [PATCH 04/34] Add unit tests for SecuritySanityCheckFilter and PreResponseAuthorizationCheckFilter --- .../server/security/NoopAuthenticator.java | 2 +- .../security/SecuritySanityCheckFilter.java | 2 +- ...eResponseAuthorizationCheckFilterTest.java | 137 ++++++++++++++++++ .../SecuritySanityCheckFilterTest.java | 85 +++++++++++ 4 files changed, 224 insertions(+), 2 deletions(-) create mode 100644 server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java create mode 100644 server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java diff --git a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java index b892ac8165b9..42bb491fef7c 100644 --- a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java +++ b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java @@ -101,7 +101,7 @@ public void destroy() @Override public String getAuthChallengeHeader() { - return null; + return "noop"; } @Override diff --git a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java index 51867dd0a603..6e8f48f6f738 100644 --- a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java @@ -72,7 +72,7 @@ public void doFilter( // make sure the original request isn't trying to fake the auth token checks Boolean authInfoChecked = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); - Boolean authToken = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); + String authToken = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); if (authInfoChecked != null || authToken != null) { sendJsonError(httpResponse, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); return; diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java new file mode 100644 index 000000000000..b0730b3a626d --- /dev/null +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -0,0 +1,137 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.http.security; + +import com.google.common.collect.Lists; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.NoopAuthenticator; +import io.druid.server.security.PreResponseAuthorizationCheckFilter; +import org.easymock.EasyMock; +import org.junit.Test; + +import javax.servlet.FilterChain; +import javax.servlet.ServletOutputStream; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.util.List; + +public class PreResponseAuthorizationCheckFilterTest +{ + private static List authenticators = Lists.newArrayList(new NoopAuthenticator()); + private static AuthConfig authConfig = new AuthConfig(true, null, null, null); + + @Test + public void testValidRequest() + { + try { + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); + EasyMock.replay(req, resp, filterChain, outputStream); + + PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( + authConfig, + authenticators, + new DefaultObjectMapper() + ); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Test + public void testAuthenticationFailedRequest() + { + try { + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn(null).once(); + resp.addHeader("WWW-Authenticate", "noop"); + EasyMock.expectLastCall().once(); + resp.setStatus(401); + EasyMock.expectLastCall().once(); + resp.setContentType("application/json"); + EasyMock.expectLastCall().once(); + resp.setCharacterEncoding("UTF-8"); + EasyMock.expectLastCall().once(); + EasyMock.replay(req, resp, filterChain, outputStream); + + PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( + authConfig, + authenticators, + new DefaultObjectMapper() + ); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Test + public void testMissingAuthorizationCheck() + { + try { + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); + EasyMock.expect(resp.getStatus()).andReturn(200).once(); + EasyMock.expect(req.getRequestURI()).andReturn("uri").once(); + resp.setStatus(403); + EasyMock.expectLastCall().once(); + resp.setContentType("application/json"); + EasyMock.expectLastCall().once(); + resp.setCharacterEncoding("UTF-8"); + EasyMock.expectLastCall().once(); + EasyMock.replay(req, resp, filterChain, outputStream); + + PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( + authConfig, + authenticators, + new DefaultObjectMapper() + ); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + } +} diff --git a/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java new file mode 100644 index 000000000000..cbdd83f30fb5 --- /dev/null +++ b/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java @@ -0,0 +1,85 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.http.security; + +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.SecuritySanityCheckFilter; +import org.easymock.EasyMock; +import org.junit.Test; + +import javax.servlet.FilterChain; +import javax.servlet.ServletOutputStream; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + +public class SecuritySanityCheckFilterTest +{ + @Test + public void testValidRequest() + { + try { + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); + + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn(null).once(); + filterChain.doFilter(req, resp); + EasyMock.expectLastCall().once(); + EasyMock.replay(req, filterChain); + SecuritySanityCheckFilter filter = new SecuritySanityCheckFilter(new DefaultObjectMapper()); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, filterChain); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Test + public void testInvalidRequest() + { + try { + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("does-not-belong").once(); + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + resp.setStatus(403); + EasyMock.expectLastCall().once(); + resp.setContentType("application/json"); + EasyMock.expectLastCall().once(); + resp.setCharacterEncoding("UTF-8"); + EasyMock.expectLastCall().once(); + + EasyMock.replay(req, resp, filterChain, outputStream); + SecuritySanityCheckFilter filter = new SecuritySanityCheckFilter(new DefaultObjectMapper()); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + } +} From 2a3aaf3125246c3f925a1f055ed68d4274387279 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 27 Jul 2017 14:37:06 -0700 Subject: [PATCH 05/34] Use Set for auth caching, close outputstreams in filters --- .../server/security/AuthorizationUtils.java | 30 +++--- .../PreResponseAuthorizationCheckFilter.java | 1 + .../security/SecuritySanityCheckFilter.java | 2 + ...eResponseAuthorizationCheckFilterTest.java | 98 ++++++++----------- .../SecuritySanityCheckFilterTest.java | 70 ++++++------- 5 files changed, 92 insertions(+), 109 deletions(-) diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index 915efd877a7d..9432d7c539a5 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import io.druid.java.util.common.ISE; import javax.servlet.http.HttpServletRequest; @@ -29,6 +30,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Set; /** * Static utility functions for performing authorization checks. @@ -100,10 +102,11 @@ public static Access authorizeAllResourceActions( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } - final Map resultCache = Maps.newHashMap(); + // this method returns on first failure, so only successful Access results are kept in the cache + final Set resultCache = Sets.newHashSet(); + for (ResourceAction resourceAction : resourceActions) { - // this method returns on first failure, so only successful Access results are kept in the cache - if (resultCache.get(resourceAction) != null) { + if (resultCache.contains(resourceAction)) { continue; } final Access access = authorizationManager.authorize( @@ -115,7 +118,7 @@ public static Access authorizeAllResourceActions( request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); return access; } else { - resultCache.put(resourceAction, access); + resultCache.add(resourceAction); } } @@ -164,11 +167,12 @@ public static Access authorizeAllResourceActions( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } - final Map resultCache = Maps.newHashMap(); + // this method returns on first failure, so only successful Access results are kept in the cache + final Set resultCache = Sets.newHashSet(); + for (ResType resource : resources) { final ResourceAction resourceAction = raGenerator.apply(resource); - // this method returns on first failure, so only successful Access results are kept in the cache - if (resultCache.get(resourceAction) != null) { + if (resultCache.contains(resourceAction)) { continue; } final Access access = authorizationManager.authorize( @@ -180,7 +184,7 @@ public static Access authorizeAllResourceActions( request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); return access; } else { - resultCache.put(resourceAction, access); + resultCache.add(resourceAction); } } @@ -218,11 +222,12 @@ public static Access authorizeAllResourceActions( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } - final Map resultCache = Maps.newHashMap(); + // this method returns on first failure, so only successful Access results are kept in the cache + final Set resultCache = Sets.newHashSet(); + for (ResType resource : resources) { final ResourceAction resourceAction = raGenerator.apply(resource); - // this method returns on first failure, so only successful Access results are kept in the cache - if (resultCache.get(resourceAction) != null) { + if (resultCache.contains(resourceAction)) { continue; } final Access access = authorizationManager.authorize( @@ -233,7 +238,7 @@ public static Access authorizeAllResourceActions( if (!access.isAllowed()) { return access; } else { - resultCache.put(resourceAction, access); + resultCache.add(resourceAction); } } @@ -313,7 +318,6 @@ public ResourceAction apply(String input) } }; - /** * Function for the common pattern of generating a resource-action for reading from a datasource, using the * datasource name. diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index 045eec6a7a37..b9edba98a03f 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -111,6 +111,7 @@ public void doFilter( response.addHeader("WWW-Authenticate", authScheme); } sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); + out.close(); return; } diff --git a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java index 6e8f48f6f738..4617837011be 100644 --- a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java @@ -75,9 +75,11 @@ public void doFilter( String authToken = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); if (authInfoChecked != null || authToken != null) { sendJsonError(httpResponse, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); + out.close(); return; } + out.close(); chain.doFilter(request, response); } diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index b0730b3a626d..7292178f98a9 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -40,36 +40,30 @@ public class PreResponseAuthorizationCheckFilterTest private static AuthConfig authConfig = new AuthConfig(true, null, null, null); @Test - public void testValidRequest() + public void testValidRequest() throws Exception { - try { - HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); - HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); - FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); - ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); - EasyMock.replay(req, resp, filterChain, outputStream); + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); + EasyMock.replay(req, resp, filterChain, outputStream); - PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( - authConfig, - authenticators, - new DefaultObjectMapper() - ); - filter.doFilter(req, resp, filterChain); - EasyMock.verify(req, resp, filterChain, outputStream); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } + PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( + authConfig, + authenticators, + new DefaultObjectMapper() + ); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); } @Test - public void testAuthenticationFailedRequest() + public void testAuthenticationFailedRequest() throws Exception { - try { HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); @@ -94,44 +88,36 @@ public void testAuthenticationFailedRequest() ); filter.doFilter(req, resp, filterChain); EasyMock.verify(req, resp, filterChain, outputStream); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } } @Test - public void testMissingAuthorizationCheck() + public void testMissingAuthorizationCheck() throws Exception { - try { - HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); - HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); - FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); - ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); - EasyMock.expect(resp.getStatus()).andReturn(200).once(); - EasyMock.expect(req.getRequestURI()).andReturn("uri").once(); - resp.setStatus(403); - EasyMock.expectLastCall().once(); - resp.setContentType("application/json"); - EasyMock.expectLastCall().once(); - resp.setCharacterEncoding("UTF-8"); - EasyMock.expectLastCall().once(); - EasyMock.replay(req, resp, filterChain, outputStream); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( - authConfig, - authenticators, - new DefaultObjectMapper() - ); - filter.doFilter(req, resp, filterChain); - EasyMock.verify(req, resp, filterChain, outputStream); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); + EasyMock.expect(resp.getStatus()).andReturn(200).once(); + EasyMock.expect(req.getRequestURI()).andReturn("uri").once(); + resp.setStatus(403); + EasyMock.expectLastCall().once(); + resp.setContentType("application/json"); + EasyMock.expectLastCall().once(); + resp.setCharacterEncoding("UTF-8"); + EasyMock.expectLastCall().once(); + EasyMock.replay(req, resp, filterChain, outputStream); + + PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( + authConfig, + authenticators, + new DefaultObjectMapper() + ); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); } } diff --git a/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java index cbdd83f30fb5..8247560bb1d2 100644 --- a/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java @@ -33,53 +33,43 @@ public class SecuritySanityCheckFilterTest { @Test - public void testValidRequest() + public void testValidRequest() throws Exception { - try { - HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); - HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); - FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn(null).once(); - filterChain.doFilter(req, resp); - EasyMock.expectLastCall().once(); - EasyMock.replay(req, filterChain); - SecuritySanityCheckFilter filter = new SecuritySanityCheckFilter(new DefaultObjectMapper()); - filter.doFilter(req, resp, filterChain); - EasyMock.verify(req, filterChain); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn(null).once(); + filterChain.doFilter(req, resp); + EasyMock.expectLastCall().once(); + EasyMock.replay(req, filterChain); + SecuritySanityCheckFilter filter = new SecuritySanityCheckFilter(new DefaultObjectMapper()); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, filterChain); } @Test - public void testInvalidRequest() + public void testInvalidRequest() throws Exception { - try { - HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); - HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); - FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); - ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("does-not-belong").once(); - EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - resp.setStatus(403); - EasyMock.expectLastCall().once(); - resp.setContentType("application/json"); - EasyMock.expectLastCall().once(); - resp.setCharacterEncoding("UTF-8"); - EasyMock.expectLastCall().once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("does-not-belong").once(); + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + resp.setStatus(403); + EasyMock.expectLastCall().once(); + resp.setContentType("application/json"); + EasyMock.expectLastCall().once(); + resp.setCharacterEncoding("UTF-8"); + EasyMock.expectLastCall().once(); - EasyMock.replay(req, resp, filterChain, outputStream); - SecuritySanityCheckFilter filter = new SecuritySanityCheckFilter(new DefaultObjectMapper()); - filter.doFilter(req, resp, filterChain); - EasyMock.verify(req, resp, filterChain, outputStream); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } + EasyMock.replay(req, resp, filterChain, outputStream); + SecuritySanityCheckFilter filter = new SecuritySanityCheckFilter(new DefaultObjectMapper()); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); } } From 28129e21f113f7a8c7fdcf5ba8bee0650dfbd941 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 27 Jul 2017 15:19:37 -0700 Subject: [PATCH 06/34] Don't close output stream on success in sanity check filter --- .../java/io/druid/server/security/SecuritySanityCheckFilter.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java index 4617837011be..0ae79969f3f3 100644 --- a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java @@ -79,7 +79,6 @@ public void doFilter( return; } - out.close(); chain.doFilter(request, response); } From d3d5c5976a2a44754f0e680b086ef4fb5a00e94b Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 27 Jul 2017 16:18:30 -0700 Subject: [PATCH 07/34] Add ConfigResourceFilter to coordinator lookups --- server/src/main/java/io/druid/query/lookup/LookupModule.java | 3 +++ .../java/io/druid/server/http/LookupCoordinatorResource.java | 3 +++ 2 files changed, 6 insertions(+) diff --git a/server/src/main/java/io/druid/query/lookup/LookupModule.java b/server/src/main/java/io/druid/query/lookup/LookupModule.java index 51b7ba9b36ae..0a0722ab9325 100644 --- a/server/src/main/java/io/druid/query/lookup/LookupModule.java +++ b/server/src/main/java/io/druid/query/lookup/LookupModule.java @@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Inject; +import com.sun.jersey.spi.container.ResourceFilters; import io.druid.common.utils.ServletResourceUtils; import io.druid.curator.announcement.Announcer; import io.druid.guice.ExpressionModule; @@ -47,6 +48,7 @@ import io.druid.query.expression.LookupExprMacro; import io.druid.server.DruidNode; import io.druid.server.http.HostAndPortWithScheme; +import io.druid.server.http.security.ConfigResourceFilter; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.initialization.jetty.JettyBindings; import io.druid.server.listener.announcer.ListenerResourceAnnouncer; @@ -104,6 +106,7 @@ public void configure(Binder binder) } @Path(ListenerResource.BASE_PATH + "/" + LookupCoordinatorManager.LOOKUP_LISTEN_ANNOUNCE_KEY) +@ResourceFilters(ConfigResourceFilter.class) class LookupListeningResource extends ListenerResource { private static final Logger LOG = new Logger(LookupListeningResource.class); diff --git a/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java b/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java index 112d85cd9b7b..aeadb3553b33 100644 --- a/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java +++ b/server/src/main/java/io/druid/server/http/LookupCoordinatorResource.java @@ -28,6 +28,7 @@ import com.google.common.base.Strings; import com.google.common.net.HostAndPort; import com.google.inject.Inject; +import com.sun.jersey.spi.container.ResourceFilters; import io.druid.audit.AuditInfo; import io.druid.audit.AuditManager; import io.druid.common.utils.ServletResourceUtils; @@ -37,6 +38,7 @@ import io.druid.java.util.common.RE; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.LookupsState; +import io.druid.server.http.security.ConfigResourceFilter; import io.druid.server.lookup.cache.LookupCoordinatorManager; import io.druid.server.lookup.cache.LookupExtractorFactoryMapContainer; @@ -67,6 +69,7 @@ * Contains information about lookups exposed through the coordinator */ @Path("/druid/coordinator/v1/lookups") +@ResourceFilters(ConfigResourceFilter.class) public class LookupCoordinatorResource { private static final Logger LOG = new Logger(LookupCoordinatorResource.class); From 1f1dabad3f1de36aea3a6574c7a62f0e56d72da3 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 27 Jul 2017 19:50:43 -0700 Subject: [PATCH 08/34] Fix filtering authorization check for empty resource list --- .../src/main/java/io/druid/server/QueryLifecycleFactory.java | 2 +- .../main/java/io/druid/server/security/AuthorizationUtils.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java index 921964efeea2..4318ddcd6280 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java +++ b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java @@ -52,7 +52,7 @@ public QueryLifecycleFactory( final ServerConfig serverConfig, final AuthConfig authConfig, final AuthorizationManagerMapper authorizationManagerMapper - ) + ) { this.warehouse = warehouse; this.texasRanger = texasRanger; diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index 9432d7c539a5..9f6e4fff3a02 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -280,6 +280,7 @@ public static List filterAuthorizedResources( throw new ISE("No authorization manager found for namespace: [%s].", namespace); } + int initialSize = resources.size(); final Map resultCache = Maps.newHashMap(); List filteredResources = new ArrayList<>(); for (ResType resource : resources) { @@ -298,7 +299,7 @@ public static List filterAuthorizedResources( } } - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, filteredResources.size() > 0); + request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, (filteredResources.size() > 0 || initialSize == 0)); return filteredResources; } From a632f4bc99065be79af10ccc024618f80f33faf3 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 28 Jul 2017 14:42:54 -0700 Subject: [PATCH 09/34] HttpClient users must explicitly escalate the client --- .../kerberos/KerberosAuthenticator.java | 2 +- .../kafka/KafkaIndexTaskClientFactory.java | 9 +- .../kafka/supervisor/KafkaSupervisorTest.java | 7 +- .../dependency-reduced-pom.xml | 210 ++++++++++++++++++ .../RemoteTaskActionClientFactory.java | 6 +- .../overlord/RemoteTaskRunnerFactory.java | 8 +- .../overlord/RemoteTaskRunnerFactoryTest.java | 5 +- .../io/druid/client/BrokerServerView.java | 4 +- ...lteredHttpServerInventoryViewProvider.java | 8 +- .../druid/client/HttpServerInventoryView.java | 8 +- .../HttpServerInventoryViewProvider.java | 8 +- .../client/coordinator/CoordinatorClient.java | 6 +- .../indexing/IndexingServiceClient.java | 6 +- .../http/AbstractHttpClientProvider.java | 18 -- .../io/druid/guice/http/HttpClientModule.java | 20 +- .../druid/initialization/Initialization.java | 2 + .../AuthenticatorHttpClientWrapperModule.java | 72 ++++++ .../server/log/StartupLoggingConfig.java | 2 +- .../cache/LookupCoordinatorManager.java | 12 +- .../server/router/CoordinatorRuleManager.java | 6 +- .../druid/server/security/Authenticator.java | 2 +- .../AuthenticatorHttpClientWrapper.java | 80 +++++++ .../server/security/NoopAuthenticator.java | 2 +- .../PreResponseAuthorizationCheckFilter.java | 20 +- .../io/druid/client/BrokerServerViewTest.java | 3 + .../DruidCoordinatorSegmentMergerTest.java | 9 +- .../cache/LookupCoordinatorManagerTest.java | 69 ++++-- .../router/TieredBrokerHostSelectorTest.java | 10 +- 28 files changed, 525 insertions(+), 89 deletions(-) create mode 100644 extensions-core/protobuf-extensions/dependency-reduced-pom.xml create mode 100644 server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java create mode 100644 server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index d5feefe686ec..a5efa6be0af1 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -416,7 +416,7 @@ public boolean authenticateJDBCContext(Map context) } @Override - public HttpClient createInternalClient(HttpClient baseClient) + public HttpClient createEscalatedClient(HttpClient baseClient) { return new KerberosHttpClient(baseClient, authConfig); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index ee602e50dbfc..41bb5602e45f 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -25,6 +25,7 @@ import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Json; import io.druid.indexing.common.TaskInfoProvider; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.joda.time.Duration; public class KafkaIndexTaskClientFactory @@ -33,9 +34,13 @@ public class KafkaIndexTaskClientFactory private ObjectMapper mapper; @Inject - public KafkaIndexTaskClientFactory(@Global HttpClient httpClient, @Json ObjectMapper mapper) + public KafkaIndexTaskClientFactory( + @Global HttpClient httpClient, + @Json ObjectMapper mapper, + AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + ) { - this.httpClient = httpClient; + this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); this.mapper = mapper; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 39470aa60b72..a23fb13b8463 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -69,6 +69,7 @@ import io.druid.server.metrics.DruidMonitorSchedulerConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -1750,7 +1751,11 @@ private KafkaSupervisor getSupervisor( skipOffsetGaps ); - KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory(null, null) + KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( + null, + null, + new AuthenticatorHttpClientWrapper(new AuthConfig(), null) + ) { @Override public KafkaIndexTaskClient build( diff --git a/extensions-core/protobuf-extensions/dependency-reduced-pom.xml b/extensions-core/protobuf-extensions/dependency-reduced-pom.xml new file mode 100644 index 000000000000..7fca520cfbf0 --- /dev/null +++ b/extensions-core/protobuf-extensions/dependency-reduced-pom.xml @@ -0,0 +1,210 @@ + + + + druid + io.druid + 0.11.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + io.druid.extensions + druid-protobuf-extensions + druid-protobuf-extensions + druid-protobuf-extensions + + + + maven-shade-plugin + 3.0.0 + + + package + + shade + + + + + + + com.google.protobuf + shaded.com.google.protobuf + + + + + + maven-resources-plugin + 3.0.2 + + + desc + + + + + + + + strict + + + + maven-compiler-plugin + + + -Xep:MissingOverride:WARN + + + + + + + + + + io.druid + druid-common + 0.11.0-SNAPSHOT + provided + + + java-util + io.druid + + + druid-api + io.druid + + + commons-codec + commons-codec + + + commons-lang + commons-lang + + + commons-dbcp2 + org.apache.commons + + + commons-pool + commons-pool + + + config-magic + org.skife.config + + + hibernate-validator + org.hibernate + + + javax.el-api + javax.el + + + validation-api + javax.validation + + + jackson-core + com.fasterxml.jackson.core + + + jackson-annotations + com.fasterxml.jackson.core + + + jackson-databind + com.fasterxml.jackson.core + + + jackson-datatype-guava + com.fasterxml.jackson.datatype + + + jackson-datatype-joda + com.fasterxml.jackson.datatype + + + jackson-dataformat-smile + com.fasterxml.jackson.dataformat + + + guice + com.google.inject + + + guice-multibindings + com.google.inject.extensions + + + jdbi + org.jdbi + + + joda-time + joda-time + + + jsr305 + com.google.code.findbugs + + + log4j-api + org.apache.logging.log4j + + + log4j-core + org.apache.logging.log4j + + + log4j-slf4j-impl + org.apache.logging.log4j + + + log4j-jul + org.apache.logging.log4j + + + log4j-1.2-api + org.apache.logging.log4j + + + jcl-over-slf4j + org.slf4j + + + disruptor + com.lmax + + + jets3t + net.java.dev.jets3t + + + antlr4-runtime + org.antlr + + + + + junit + junit + 4.11 + test + + + hamcrest-core + org.hamcrest + + + + + + 3.2.0 + + + diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java index 2556621db1f0..8d7a9844c7b0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java @@ -27,6 +27,7 @@ import io.druid.guice.annotations.Global; import io.druid.indexing.common.RetryPolicyFactory; import io.druid.indexing.common.task.Task; +import io.druid.server.security.AuthenticatorHttpClientWrapper; /** */ @@ -42,10 +43,11 @@ public RemoteTaskActionClientFactory( @Global HttpClient httpClient, @IndexingService ServerDiscoverySelector selector, RetryPolicyFactory retryPolicyFactory, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper ) { - this.httpClient = httpClient; + this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); this.selector = selector; this.retryPolicyFactory = retryPolicyFactory; this.jsonMapper = jsonMapper; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java index d0e621ae592f..e18517a7b2d0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java @@ -32,6 +32,7 @@ import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.server.initialization.IndexerZkConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.apache.curator.framework.CuratorFramework; /** @@ -59,14 +60,15 @@ public RemoteTaskRunnerFactory( final Supplier workerConfigRef, final ScheduledExecutorFactory factory, final ProvisioningSchedulerConfig provisioningSchedulerConfig, - final ProvisioningStrategy provisioningStrategy - ) + final ProvisioningStrategy provisioningStrategy, + final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + ) { this.curator = curator; this.remoteTaskRunnerConfig = remoteTaskRunnerConfig; this.zkPaths = zkPaths; this.jsonMapper = jsonMapper; - this.httpClient = httpClient; + this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); this.workerConfigRef = workerConfigRef; this.provisioningSchedulerConfig = provisioningSchedulerConfig; this.provisioningStrategy = provisioningStrategy; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerFactoryTest.java index e97d92812033..32d70518786a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerFactoryTest.java @@ -35,6 +35,8 @@ import io.druid.java.util.common.concurrent.ScheduledExecutors; import io.druid.server.initialization.IndexerZkConfig; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -131,7 +133,8 @@ public boolean isDoAutoscale() provisioningConfig, workerBehaviorConfig, provisioningSchedulerConfig - ) + ), + new AuthenticatorHttpClientWrapper(new AuthConfig(), null) ); Assert.assertEquals(0, executorCount.get()); RemoteTaskRunner remoteTaskRunner1 = factory.build(); diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index 13495c552654..36024a5aa5bc 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -40,6 +40,7 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -83,6 +84,7 @@ public BrokerServerView( QueryWatcher queryWatcher, @Smile ObjectMapper smileMapper, @Client HttpClient httpClient, + AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper, FilteredServerInventoryView baseView, TierSelectorStrategy tierSelectorStrategy, ServiceEmitter emitter, @@ -92,7 +94,7 @@ public BrokerServerView( this.warehouse = warehouse; this.queryWatcher = queryWatcher; this.smileMapper = smileMapper; - this.httpClient = httpClient; + this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); this.baseView = baseView; this.tierSelectorStrategy = tierSelectorStrategy; this.emitter = emitter; diff --git a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java index 6dbb7cf4bcb6..80d83e886aec 100644 --- a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.Pair; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; @@ -65,6 +66,10 @@ public class FilteredHttpServerInventoryViewProvider implements FilteredServerIn @NotNull private CuratorFramework curator = null; + @JacksonInject + @NotNull + private AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = null; + @Override public HttpServerInventoryView get() { @@ -72,7 +77,8 @@ public HttpServerInventoryView get() jsonMapper, smileMapper, httpClient, new DruidServerDiscovery(curator, zkPaths.getAnnouncementsPath(), jsonMapper), Predicates.>alwaysTrue(), - config + config, + authenticatorHttpClientWrapper ); } } diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryView.java b/server/src/main/java/io/druid/client/HttpServerInventoryView.java index 9419f55d0fb1..02f0d64032e1 100644 --- a/server/src/main/java/io/druid/client/HttpServerInventoryView.java +++ b/server/src/main/java/io/druid/client/HttpServerInventoryView.java @@ -54,6 +54,7 @@ import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.coordination.SegmentChangeRequestHistory; import io.druid.server.coordination.SegmentChangeRequestsSnapshot; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -118,10 +119,11 @@ public HttpServerInventoryView( final @Global HttpClient httpClient, final DruidServerDiscovery serverDiscovery, final Predicate> defaultFilter, - final HttpServerInventoryViewConfig config - ) + final HttpServerInventoryViewConfig config, + final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + ) { - this.httpClient = httpClient; + this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); this.smileMapper = smileMapper; this.serverDiscovery = serverDiscovery; this.defaultFilter = defaultFilter; diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java index efc109542ec2..a484e3c40cb6 100644 --- a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.Pair; import io.druid.server.coordination.DruidServerMetadata; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; @@ -65,6 +66,10 @@ public class HttpServerInventoryViewProvider implements ServerInventoryViewProvi @NotNull private CuratorFramework curator = null; + @JacksonInject + @NotNull + private AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = null; + @Override public HttpServerInventoryView get() { @@ -74,7 +79,8 @@ public HttpServerInventoryView get() httpClient, new DruidServerDiscovery(curator, zkPaths.getAnnouncementsPath(), jsonMapper), Predicates.>alwaysTrue(), - config + config, + authenticatorHttpClientWrapper ); } } diff --git a/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java index b70da3e5d75a..b4dff7e9bb62 100644 --- a/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java @@ -35,6 +35,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Interval; @@ -55,10 +56,11 @@ public class CoordinatorClient public CoordinatorClient( @Global HttpClient client, ObjectMapper jsonMapper, - @Coordinator ServerDiscoverySelector selector + @Coordinator ServerDiscoverySelector selector, + AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper ) { - this.client = client; + this.client = authenticatorHttpClientWrapper.getEscalatedClient(client); this.jsonMapper = jsonMapper; this.selector = selector; } diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index 96b5d9bb7d9c..469a04189396 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -31,6 +31,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; @@ -54,10 +55,11 @@ public class IndexingServiceClient public IndexingServiceClient( @Global HttpClient client, ObjectMapper jsonMapper, - @IndexingService ServerDiscoverySelector selector + @IndexingService ServerDiscoverySelector selector, + AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper ) { - this.client = client; + this.client = authenticatorHttpClientWrapper.getEscalatedClient(client); this.jsonMapper = jsonMapper; this.selector = selector; } diff --git a/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java b/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java index 1f489f4c2f5d..8d1e209ca727 100644 --- a/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java +++ b/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java @@ -27,13 +27,9 @@ import com.google.inject.Provider; import com.google.inject.TypeLiteral; import io.druid.java.util.common.lifecycle.Lifecycle; -import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthenticationUtils; -import io.druid.server.security.Authenticator; import javax.net.ssl.SSLContext; import java.lang.annotation.Annotation; -import java.util.List; /** */ @@ -94,20 +90,6 @@ public Provider> getConfigProvider() { return injector.getProvider(configKey); } - - public AuthConfig getAuthConfig() - { - return injector.getInstance(AuthConfig.class); - } - - public List getAuthenticatorChain() - { - return AuthenticationUtils.getAuthenticatorChainFromConfig( - getAuthConfig().getAuthenticatorChain(), - injector - ); - } - public Provider getLifecycleProvider() { return injector.getProvider(Lifecycle.class); diff --git a/server/src/main/java/io/druid/guice/http/HttpClientModule.java b/server/src/main/java/io/druid/guice/http/HttpClientModule.java index 9bfda3e89a6f..3eb38fd892e9 100644 --- a/server/src/main/java/io/druid/guice/http/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/http/HttpClientModule.java @@ -27,13 +27,9 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.annotations.Global; -import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; -import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; import java.lang.annotation.Annotation; -import java.util.List; /** */ @@ -122,24 +118,10 @@ public HttpClient get() builder.withSslContext(getSslContextBinding().getProvider().get()); } - HttpClient client = HttpClientInit.createClient( + return HttpClientInit.createClient( builder.build(), LifecycleUtils.asMmxLifecycle(getLifecycleProvider().get()) ); - final AuthConfig authConfig = getAuthConfig(); - if (authConfig.isEnabled()) { - List authenticators = getAuthenticatorChain(); - for (Authenticator authenticator : authenticators) { - if (authenticator.getTypeName().equals(authConfig.getInternalAuthenticator())) { - return authenticator.createInternalClient(client); - } - } - throw new ISE( - "Could not locate internal authenticator with type name: %s", - authConfig.getInternalAuthenticator() - ); - } - return client; } } } diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 439055c1b96f..f1f526ebfe48 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -62,6 +62,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; +import io.druid.server.initialization.AuthenticatorHttpClientWrapperModule; import io.druid.server.initialization.AuthorizationManagerMapperModule; import io.druid.server.initialization.EmitterModule; import io.druid.server.initialization.jetty.JettyServerModule; @@ -348,6 +349,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new Log4jShutterDownerModule(), new DruidAuthModule(), new AuthenticatorModule(), + new AuthenticatorHttpClientWrapperModule(), new AuthorizationManagerModule(), new AuthorizationManagerMapperModule(), new LifecycleModule(), diff --git a/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java new file mode 100644 index 000000000000..dc5f5057e647 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java @@ -0,0 +1,72 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.initialization; + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Provider; +import io.druid.guice.LazySingleton; +import io.druid.initialization.DruidModule; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; + +import java.util.Collections; +import java.util.List; + +public class AuthenticatorHttpClientWrapperModule implements DruidModule +{ + private static Logger log = new Logger(AuthenticatorHttpClientWrapperModule.class); + + @Override + public void configure(Binder binder) + { + binder.bind(AuthenticatorHttpClientWrapper.class) + .toProvider(new AuthenticatorHttpClientWrapperProvider()) + .in(LazySingleton.class); + } + + @Override + public List getJacksonModules() + { + return Collections.EMPTY_LIST; + } + + private static class AuthenticatorHttpClientWrapperProvider implements Provider + { + private AuthConfig authConfig; + private Injector injector; + + @Inject + public void inject(Injector injector) + { + this.authConfig = injector.getInstance(AuthConfig.class); + this.injector = injector; + } + + @Override + public AuthenticatorHttpClientWrapper get() + { + return new AuthenticatorHttpClientWrapper(authConfig, injector); + } + } +} diff --git a/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java b/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java index 51776da266af..81568b4ba882 100644 --- a/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java +++ b/server/src/main/java/io/druid/server/log/StartupLoggingConfig.java @@ -30,7 +30,7 @@ public class StartupLoggingConfig private boolean logProperties = false; @JsonProperty - private List maskProperties = ImmutableList.of("password"); + private List maskProperties = ImmutableList.of("Password", "password", "Secret", "secret"); public boolean isLogProperties() { diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index 22221e6b73c2..9e9f3e689dd0 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -59,6 +59,7 @@ import io.druid.server.http.HostAndPortWithScheme; import io.druid.server.listener.announcer.ListenerDiscoverer; import io.druid.server.listener.resource.ListenerResource; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -137,14 +138,19 @@ public LookupCoordinatorManager( final ListenerDiscoverer listenerDiscoverer, final @Smile ObjectMapper smileMapper, final JacksonConfigManager configManager, - final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig - ) + final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig, + final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + ) { this( listenerDiscoverer, configManager, lookupCoordinatorManagerConfig, - new LookupsCommunicator(httpClient, lookupCoordinatorManagerConfig, smileMapper) + new LookupsCommunicator( + authenticatorHttpClientWrapper.getEscalatedClient(httpClient), + lookupCoordinatorManagerConfig, + smileMapper + ) ); } diff --git a/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java b/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java index 66bcdb0d2e56..534843afb128 100644 --- a/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java +++ b/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java @@ -40,6 +40,7 @@ import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.java.util.common.logger.Logger; import io.druid.server.coordinator.rules.Rule; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Duration; @@ -79,10 +80,11 @@ public CoordinatorRuleManager( @Global HttpClient httpClient, @Json ObjectMapper jsonMapper, Supplier config, - ServerDiscoverySelector selector + ServerDiscoverySelector selector, + AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper ) { - this.httpClient = httpClient; + this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); this.jsonMapper = jsonMapper; this.config = config; this.selector = selector; diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index 2302e930c8ec..0b62ea2ec2e4 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -108,5 +108,5 @@ public interface Authenticator extends ServletFilterHolder * * @return HttpClient that sends requests with the credentials of the internal system user */ - public HttpClient createInternalClient(HttpClient baseClient); + public HttpClient createEscalatedClient(HttpClient baseClient); } diff --git a/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java b/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java new file mode 100644 index 000000000000..5c0d8e09599f --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java @@ -0,0 +1,80 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.google.common.base.Preconditions; +import com.google.inject.Injector; +import com.metamx.http.client.HttpClient; +import io.druid.java.util.common.ISE; + +import java.util.List; + +/** + * Singleton utility object that creates escalated HttpClients using a configuration-specified Authenticator's + * getEscalatedClient() method. + */ +public class AuthenticatorHttpClientWrapper +{ + private final AuthConfig authConfig; + private Authenticator internalAuthenticator; + + public AuthenticatorHttpClientWrapper( + final AuthConfig authConfig, + final Injector injector + ) + { + this.authConfig = authConfig; + + if (authConfig.isEnabled()) { + Preconditions.checkNotNull( + authConfig.getInternalAuthenticator(), + "Auth is enabled but no internal authenticator is configured." + ); + Preconditions.checkNotNull( + authConfig.getAuthenticatorChain(), + "Auth is enabled but no authenticators have been configured." + ); + + List authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( + authConfig.getAuthenticatorChain(), + injector + ); + String internalAuthenticatorName = authConfig.getInternalAuthenticator(); + for (Authenticator authenticator : authenticators) { + if (authenticator.getTypeName().equals(internalAuthenticatorName)) { + internalAuthenticator = authenticator; + break; + } + } + if (internalAuthenticator == null) { + throw new ISE("Could not locate internal authenticator with type name: %s", internalAuthenticatorName); + } + } + } + + public HttpClient getEscalatedClient(HttpClient baseClient) + { + if (authConfig.isEnabled()) { + return internalAuthenticator.createEscalatedClient(baseClient); + } else { + return baseClient; + } + } +} diff --git a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java index 42bb491fef7c..6a1ed4e39204 100644 --- a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java +++ b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java @@ -111,7 +111,7 @@ public boolean authenticateJDBCContext(Map context) } @Override - public HttpClient createInternalClient(HttpClient baseClient) + public HttpClient createEscalatedClient(HttpClient baseClient) { return baseClient; } diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index b9edba98a03f..b291ca1b56d6 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.query.QueryInterruptedException; import io.druid.server.DruidNode; @@ -115,6 +117,21 @@ public void doFilter( return; } + filterChain.doFilter(servletRequest, servletResponse); + + authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); + if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { + String errorMsg = StringUtils.format( + "Request did not have an authorization check performed: %s", + ((HttpServletRequest) servletRequest).getRequestURI() + ); + log.error(errorMsg); + throw new ISE(errorMsg); + } + + /* + this breaks proxying, disable for now. + // capture the response stream before its sent to client, or we don't get a chance to modify it later // http://www.oracle.com/technetwork/java/filters-137243.html GenericResponseWrapper wrapper = new GenericResponseWrapper((HttpServletResponse) servletResponse); @@ -130,10 +147,11 @@ public void doFilter( ((HttpServletRequest) servletRequest).getRequestURI() ); sendJsonError(response, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); + out.close(); } else { out.write(wrapper.getData()); } - out.close(); + */ } else { filterChain.doFilter(servletRequest, servletResponse); } diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index b16c6684f458..bf08e2a47ec4 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -42,6 +42,8 @@ import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -336,6 +338,7 @@ public CallbackAction segmentViewInitialized() EasyMock.createMock(QueryWatcher.class), getSmileMapper(), EasyMock.createMock(HttpClient.class), + new AuthenticatorHttpClientWrapper(new AuthConfig(), null), baseView, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), new NoopServiceEmitter(), diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java index fc541c58e025..8ce29d40b2b6 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java @@ -26,6 +26,8 @@ import io.druid.client.indexing.IndexingServiceClient; import io.druid.common.config.JacksonConfigManager; import io.druid.server.coordinator.helper.DruidCoordinatorSegmentMerger; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import org.easymock.EasyMock; @@ -455,7 +457,12 @@ private static List> merge(final Collection segme EasyMock.replay(configManager); final List> retVal = Lists.newArrayList(); - final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null) + final IndexingServiceClient indexingServiceClient = new IndexingServiceClient( + null, + null, + null, + new AuthenticatorHttpClientWrapper(new AuthConfig(), null) + ) { @Override public void mergeSegments(List segmentsToMerge) diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 6a9dde6bd735..044084993eee 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -44,6 +44,8 @@ import io.druid.query.lookup.LookupsState; import io.druid.server.http.HostAndPortWithScheme; import io.druid.server.listener.announcer.ListenerDiscoverer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.easymock.EasyMock; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -77,6 +79,10 @@ public class LookupCoordinatorManagerTest private final HttpClient client = EasyMock.createStrictMock(HttpClient.class); private final JacksonConfigManager configManager = EasyMock.createStrictMock(JacksonConfigManager.class); private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig = new LookupCoordinatorManagerConfig(); + private final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = new AuthenticatorHttpClientWrapper( + new AuthConfig(), + null + ); private static final String LOOKUP_TIER = "lookup_tier"; private static final String SINGLE_LOOKUP_NAME = "lookupName"; @@ -535,7 +541,8 @@ public void testUpdateLookupsFailsUnitialized() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -558,7 +565,8 @@ public void testUpdateLookupsInitialization() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -590,7 +598,8 @@ public void testUpdateLookupAdds() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -627,7 +636,8 @@ public void testUpdateLookupsAddsNewLookup() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -689,7 +699,8 @@ public void testUpdateLookupsOnlyUpdatesToTier() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -744,7 +755,8 @@ public void testUpdateLookupsUpdates() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -774,7 +786,8 @@ public void testUpdateLookupFailsSameVersionUpdates() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -808,7 +821,8 @@ public void testUpdateLookupsAddsNewTier() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -863,7 +877,8 @@ public void testDeleteLookup() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -908,7 +923,8 @@ public void testDeleteLookupIgnoresMissing() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -933,7 +949,8 @@ public void testDeleteLookupIgnoresNotReady() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -959,7 +976,8 @@ public void testGetLookup() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -988,7 +1006,8 @@ public void testGetLookupIgnoresMalformed() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -1013,7 +1032,8 @@ public void testGetLookupIgnoresNotReady() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ) { @Override @@ -1166,7 +1186,8 @@ public void testGetToBeLoadedOnNode() discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ); LookupsState currNodeState = new LookupsState<>( @@ -1202,7 +1223,8 @@ public void testToBeDropped() discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ); LookupsState currNodeState = new LookupsState<>( @@ -1249,7 +1271,8 @@ public void testStartStop() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ); Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); @@ -1286,7 +1309,8 @@ public void testMultipleStartStop() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ); Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); @@ -1338,7 +1362,8 @@ public void testLookupDiscoverAll() throws Exception discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ); manager.start(); Assert.assertEquals(fakeChildren, manager.discoverTiers()); @@ -1375,7 +1400,8 @@ public void describeTo(Description description) discoverer, mapper, configManager, - lookupCoordinatorManagerConfig + lookupCoordinatorManagerConfig, + authenticatorHttpClientWrapper ); manager.start(); @@ -1444,7 +1470,8 @@ public long getPeriod() { return 1; } - } + }, + authenticatorHttpClientWrapper ); manager.start(); EasyMock.verify(configManager); diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index 52d3808e9b1a..1451d97c9a1f 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -34,6 +34,8 @@ import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.server.coordinator.rules.IntervalLoadRule; import io.druid.server.coordinator.rules.Rule; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.After; @@ -249,7 +251,13 @@ public TestRuleManager( ServerDiscoverySelector selector ) { - super(httpClient, jsonMapper, config, selector); + super( + httpClient, + jsonMapper, + config, + selector, + new AuthenticatorHttpClientWrapper(new AuthConfig(), null) + ); } @Override From 0ef07c8c6b7b6f783ff4963be3f9d70c712b6207 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 31 Jul 2017 13:49:39 -0700 Subject: [PATCH 10/34] Remove response modification from PreResponseAuthorizationCheckFilter --- .../PreResponseAuthorizationCheckFilter.java | 136 +----------------- 1 file changed, 3 insertions(+), 133 deletions(-) diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index b291ca1b56d6..9b2e77a24171 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -32,20 +32,12 @@ import javax.servlet.FilterChain; import javax.servlet.FilterConfig; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; import javax.servlet.ServletRequest; import javax.servlet.ServletResponse; -import javax.servlet.WriteListener; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; -import javax.servlet.http.HttpServletResponseWrapper; -import java.io.BufferedWriter; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Set; @@ -125,33 +117,12 @@ public void doFilter( "Request did not have an authorization check performed: %s", ((HttpServletRequest) servletRequest).getRequestURI() ); + // Note: rather than throwing an exception here, it would be nice to blank out the original response + // since the request didn't have any authorization checks performed. However, this breaks proxying + // (e.g. OverlordServletProxy), so this is not implemented for now. log.error(errorMsg); throw new ISE(errorMsg); } - - /* - this breaks proxying, disable for now. - - // capture the response stream before its sent to client, or we don't get a chance to modify it later - // http://www.oracle.com/technetwork/java/filters-137243.html - GenericResponseWrapper wrapper = new GenericResponseWrapper((HttpServletResponse) servletResponse); - filterChain.doFilter(servletRequest, wrapper); - - // After response has been generated, something in the request processing path must have set - // DRUID_AUTH_TOKEN_CHECKED (i.e. performed an authorization check). If this is not set, - // a 403 error will be returned instead of the response. - authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); - if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { - log.error( - "Request did not have an authorization check performed: %s", - ((HttpServletRequest) servletRequest).getRequestURI() - ); - sendJsonError(response, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); - out.close(); - } else { - out.write(wrapper.getData()); - } - */ } else { filterChain.doFilter(servletRequest, servletResponse); } @@ -180,105 +151,4 @@ public static void sendJsonError(HttpServletResponse resp, int error, String err log.error("WTF? Can't get writer from HTTP response."); } } - - // classes from "Servlet Filters and Event Listeners" - // https://docs.oracle.com/cd/B14099_19/web.1012/b14017/filters.htm - private static class GenericResponseWrapper extends HttpServletResponseWrapper - { - private ByteArrayOutputStream output; - private int contentLength; - private String contentType; - - public GenericResponseWrapper(HttpServletResponse response) - { - super(response); - output = new ByteArrayOutputStream(); - } - - public byte[] getData() - { - return output.toByteArray(); - } - - @Override - public ServletOutputStream getOutputStream() - { - return new FilterServletOutputStream(output); - } - - @Override - public PrintWriter getWriter() - { - return new PrintWriter( - new BufferedWriter(new OutputStreamWriter(getOutputStream(), StandardCharsets.UTF_8)), - true - ); - } - - @Override - public void setContentLength(int length) - { - this.contentLength = length; - super.setContentLength(length); - } - - public int getContentLength() - { - return contentLength; - } - - @Override - public void setContentType(String type) - { - this.contentType = type; - super.setContentType(type); - } - - @Override - public String getContentType() - { - return contentType; - } - } - - private static class FilterServletOutputStream extends ServletOutputStream - { - - private DataOutputStream stream; - - public FilterServletOutputStream(OutputStream output) - { - stream = new DataOutputStream(output); - } - - @Override - public void write(int b) throws IOException - { - stream.write(b); - } - - @Override - public void write(byte[] b) throws IOException - { - stream.write(b); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException - { - stream.write(b, off, len); - } - - @Override - public boolean isReady() - { - return false; - } - - @Override - public void setWriteListener(WriteListener writeListener) - { - - } - } } From 5c34b2dacbf96813fa6c8d2caff6dd31ad75362a Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 31 Jul 2017 13:53:11 -0700 Subject: [PATCH 11/34] Remove extraneous pom.xml --- .../dependency-reduced-pom.xml | 210 ------------------ 1 file changed, 210 deletions(-) delete mode 100644 extensions-core/protobuf-extensions/dependency-reduced-pom.xml diff --git a/extensions-core/protobuf-extensions/dependency-reduced-pom.xml b/extensions-core/protobuf-extensions/dependency-reduced-pom.xml deleted file mode 100644 index 7fca520cfbf0..000000000000 --- a/extensions-core/protobuf-extensions/dependency-reduced-pom.xml +++ /dev/null @@ -1,210 +0,0 @@ - - - - druid - io.druid - 0.11.0-SNAPSHOT - ../../pom.xml - - 4.0.0 - io.druid.extensions - druid-protobuf-extensions - druid-protobuf-extensions - druid-protobuf-extensions - - - - maven-shade-plugin - 3.0.0 - - - package - - shade - - - - - - - com.google.protobuf - shaded.com.google.protobuf - - - - - - maven-resources-plugin - 3.0.2 - - - desc - - - - - - - - strict - - - - maven-compiler-plugin - - - -Xep:MissingOverride:WARN - - - - - - - - - - io.druid - druid-common - 0.11.0-SNAPSHOT - provided - - - java-util - io.druid - - - druid-api - io.druid - - - commons-codec - commons-codec - - - commons-lang - commons-lang - - - commons-dbcp2 - org.apache.commons - - - commons-pool - commons-pool - - - config-magic - org.skife.config - - - hibernate-validator - org.hibernate - - - javax.el-api - javax.el - - - validation-api - javax.validation - - - jackson-core - com.fasterxml.jackson.core - - - jackson-annotations - com.fasterxml.jackson.core - - - jackson-databind - com.fasterxml.jackson.core - - - jackson-datatype-guava - com.fasterxml.jackson.datatype - - - jackson-datatype-joda - com.fasterxml.jackson.datatype - - - jackson-dataformat-smile - com.fasterxml.jackson.dataformat - - - guice - com.google.inject - - - guice-multibindings - com.google.inject.extensions - - - jdbi - org.jdbi - - - joda-time - joda-time - - - jsr305 - com.google.code.findbugs - - - log4j-api - org.apache.logging.log4j - - - log4j-core - org.apache.logging.log4j - - - log4j-slf4j-impl - org.apache.logging.log4j - - - log4j-jul - org.apache.logging.log4j - - - log4j-1.2-api - org.apache.logging.log4j - - - jcl-over-slf4j - org.slf4j - - - disruptor - com.lmax - - - jets3t - net.java.dev.jets3t - - - antlr4-runtime - org.antlr - - - - - junit - junit - 4.11 - test - - - hamcrest-core - org.hamcrest - - - - - - 3.2.0 - - - From ec67fd4e80157e0523070ade9dd74080fb9cd175 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 31 Jul 2017 14:45:46 -0700 Subject: [PATCH 12/34] Fix unit test --- .../security/PreResponseAuthorizationCheckFilterTest.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index 7292178f98a9..6b70162c7dc4 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -21,12 +21,15 @@ import com.google.common.collect.Lists; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.ISE; import io.druid.server.security.AuthConfig; import io.druid.server.security.Authenticator; import io.druid.server.security.NoopAuthenticator; import io.druid.server.security.PreResponseAuthorizationCheckFilter; import org.easymock.EasyMock; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import javax.servlet.FilterChain; import javax.servlet.ServletOutputStream; @@ -39,6 +42,9 @@ public class PreResponseAuthorizationCheckFilterTest private static List authenticators = Lists.newArrayList(new NoopAuthenticator()); private static AuthConfig authConfig = new AuthConfig(true, null, null, null); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + @Test public void testValidRequest() throws Exception { @@ -93,6 +99,8 @@ public void testAuthenticationFailedRequest() throws Exception @Test public void testMissingAuthorizationCheck() throws Exception { + expectedException.expect(ISE.class); + expectedException.expectMessage("Request did not have an authorization check performed: uri"); HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); From d5917f4344a3a7f65f2d8123ac52a236bd416d11 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 3 Aug 2017 18:36:30 -0700 Subject: [PATCH 13/34] Better lifecycle management --- .../AuthorizationManagerMapperModule.java | 10 +++++++--- .../io/druid/server/security/AuthConfig.java | 2 +- .../server/security/AuthorizationManager.java | 19 +++++++++++++----- .../security/AuthorizationManagerMapper.java | 20 +++++++++++++++++++ 4 files changed, 42 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java index e8df9d987bb1..b3714febbb7f 100644 --- a/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java +++ b/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java @@ -28,6 +28,7 @@ import com.google.inject.Provider; import com.google.inject.name.Names; import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleModule; import io.druid.initialization.DruidModule; import io.druid.java.util.common.logger.Logger; import io.druid.server.security.AuthConfig; @@ -49,8 +50,11 @@ public void configure(Binder binder) binder.bind(AuthorizationManagerMapper.class) .toProvider(new AuthorizationManagerMapperProvider()) .in(LazySingleton.class); + + LifecycleModule.register(binder, AuthorizationManagerMapper.class); } + @SuppressWarnings("unchecked") @Override public List getJacksonModules() { @@ -74,10 +78,10 @@ public AuthorizationManagerMapper get() { Map authorizationManagerMap = Maps.newHashMap(); - List authManagerNames = authConfig.getAuthorizationManagers(); + List authorizationManagers = authConfig.getAuthorizationManagers(); // If user didn't configure any AuthorizationManagers, use the default which rejects all requests. - if (authManagerNames == null || authManagerNames.size() == 0) { + if (authorizationManagers == null || authorizationManagers.isEmpty()) { return new AuthorizationManagerMapper(null) { @Override public AuthorizationManager getAuthorizationManager(String namespace) @@ -87,7 +91,7 @@ public AuthorizationManager getAuthorizationManager(String namespace) }; } - for (String authorizationManagerName : authManagerNames) { + for (String authorizationManagerName : authorizationManagers) { AuthorizationManager authorizationManager = injector.getInstance(Key.get( AuthorizationManager.class, Names.named(authorizationManagerName) diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index 06875338d67d..57c8f0661e10 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -73,7 +73,7 @@ public AuthConfig( private final String internalAuthenticator; @JsonProperty - List authorizationManagers; + private List authorizationManagers; public boolean isEnabled() { diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManager.java b/server/src/main/java/io/druid/server/security/AuthorizationManager.java index da1c902cf2ff..d159168d7b47 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationManager.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationManager.java @@ -43,17 +43,26 @@ public interface AuthorizationManager /** * Check if the entity represented by `identity` in `namespace` is authorized to perform `action` on `resource`. * - * @param identity The identity of the requester + * @param identity The identity of the requester * @param namespace The namespace of the identity - * @param resource The resource to be accessed - * @param action The action to perform on the resource + * @param resource The resource to be accessed + * @param action The action to perform on the resource + * * @return An Access object representing the result of the authorization check. */ - public Access authorize(String identity, Resource resource, Action action); + Access authorize(String identity, Resource resource, Action action); /** * @return The namespace associated with this AuthorizationManager. Authenticator implementations will * put the namespace in request headers. */ - public String getNamespace(); + String getNamespace(); + + default void start() + { + } + + default void stop() + { + } } diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java b/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java index cc4440750829..7caa74287b87 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java @@ -19,8 +19,12 @@ package io.druid.server.security; +import io.druid.guice.ManageLifecycle; +import io.druid.java.util.common.lifecycle.LifecycleStart; + import java.util.Map; +@ManageLifecycle public class AuthorizationManagerMapper { private Map authorizationManagerMap; @@ -36,4 +40,20 @@ public AuthorizationManager getAuthorizationManager(String namespace) { return authorizationManagerMap.get(namespace); } + + @LifecycleStart + public void start() + { + for (AuthorizationManager authorizationManager : authorizationManagerMap.values()) { + authorizationManager.start(); + } + } + + @LifecycleStart + public void stop() + { + for (AuthorizationManager authorizationManager : authorizationManagerMap.values()) { + authorizationManager.stop(); + } + } } From 427aa262a0bc5efa3c5615e7da3f97edee8e2f0e Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 7 Aug 2017 14:29:17 -0700 Subject: [PATCH 14/34] Rename AuthorizationManager to Authorizer --- docs/content/configuration/auth.md | 24 ++++----- .../druid/indexing/kafka/KafkaIndexTask.java | 10 ++-- .../overlord/http/OverlordResource.java | 16 +++--- .../security/SupervisorResourceFilter.java | 8 +-- .../http/security/TaskResourceFilter.java | 8 +-- .../supervisor/SupervisorResource.java | 14 +++--- .../io/druid/indexing/common/TestUtils.java | 4 +- .../overlord/http/OverlordResourceTest.java | 10 ++-- .../OverlordSecurityResourceFilterTest.java | 16 +++--- ...nagerModule.java => AuthorizerModule.java} | 16 +++--- .../druid/initialization/Initialization.java | 8 +-- .../io/druid/server/BrokerQueryResource.java | 6 +-- .../io/druid/server/ClientInfoResource.java | 10 ++-- .../java/io/druid/server/QueryLifecycle.java | 12 ++--- .../druid/server/QueryLifecycleFactory.java | 10 ++-- .../java/io/druid/server/QueryResource.java | 10 ++-- .../server/http/DatasourcesResource.java | 10 ++-- .../druid/server/http/IntervalsResource.java | 12 ++--- .../druid/server/http/InventoryViewUtils.java | 16 +++--- .../druid/server/http/MetadataResource.java | 10 ++-- .../http/security/AbstractResourceFilter.java | 16 +++--- .../http/security/ConfigResourceFilter.java | 8 +-- .../security/DatasourceResourceFilter.java | 8 +-- .../http/security/RulesResourceFilter.java | 8 +-- .../http/security/StateResourceFilter.java | 8 +-- ...odule.java => AuthorizerMapperModule.java} | 46 ++++++++--------- ...rModule.java => NoopAuthorizerModule.java} | 10 ++-- .../io/druid/server/security/AuthConfig.java | 22 ++++---- .../druid/server/security/Authenticator.java | 2 +- .../server/security/AuthorizationUtils.java | 50 +++++++++---------- ...horizationManager.java => Authorizer.java} | 16 +++--- ...nagerMapper.java => AuthorizerMapper.java} | 22 ++++---- ...ionManager.java => DefaultAuthorizer.java} | 4 +- ...zationManager.java => NoopAuthorizer.java} | 2 +- .../AsyncQueryForwardingServletTest.java | 14 +++--- .../io/druid/server/QueryResourceTest.java | 24 ++++----- .../server/http/DatasourcesResourceTest.java | 10 ++-- .../security/ResourceFilterTestHelper.java | 18 +++---- .../security/SecurityResourceFilterTest.java | 40 +++++++-------- .../server/initialization/JettyQosTest.java | 14 +++--- .../server/initialization/JettyTest.java | 14 +++--- .../sql/calcite/planner/DruidPlanner.java | 16 +++--- .../sql/calcite/planner/PlannerFactory.java | 10 ++-- .../sql/avatica/DruidAvaticaHandlerTest.java | 6 +-- .../druid/sql/calcite/util/CalciteTests.java | 4 +- 45 files changed, 311 insertions(+), 311 deletions(-) rename server/src/main/java/io/druid/guice/security/{AuthorizationManagerModule.java => AuthorizerModule.java} (71%) rename server/src/main/java/io/druid/server/initialization/{AuthorizationManagerMapperModule.java => AuthorizerMapperModule.java} (55%) rename server/src/main/java/io/druid/server/initialization/{NoopAuthorizationManagerModule.java => NoopAuthorizerModule.java} (81%) rename server/src/main/java/io/druid/server/security/{AuthorizationManager.java => Authorizer.java} (79%) rename server/src/main/java/io/druid/server/security/{AuthorizationManagerMapper.java => AuthorizerMapper.java} (63%) rename server/src/main/java/io/druid/server/security/{DefaultAuthorizationManager.java => DefaultAuthorizer.java} (91%) rename server/src/main/java/io/druid/server/security/{NoopAuthorizationManager.java => NoopAuthorizer.java} (93%) diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index 87d08b5782e2..1af50350fbdb 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -9,7 +9,7 @@ layout: doc_page |`druid.auth.enabled`|boolean|Determines if authentication and authorization checks will be performed on requests.|false|no| |`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|null|yes, if auth enabled| |`druid.auth.internalAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes, if auth enabled| -|`druid.auth.authorizationManagers`|JSON List of Strings|List of AuthorizationManager type names |null|yes, if auth enabled| +|`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |null|yes, if auth enabled| ## Enabling Authentication/Authorization @@ -29,27 +29,27 @@ The `druid.auth.internalAuthenticator` property determines what authentication s The Authenticator chosen for this property must also be present in `druid.auth.authenticationChain`. -## Authorization Managers -Authorization decisions are handled by an AuthorizationManager. The `druid.auth.authorizationManagers` property determines what AuthorizationManager implementations will be active. +## Authorizers +Authorization decisions are handled by an Authorizer. The `druid.auth.authorizers` property determines what Authorizer implementations will be active. -There are two built-in AuthorizationManagers, "default" and "noop". Other implementations are provided by extensions. +There are two built-in Authorizers, "default" and "noop". Other implementations are provided by extensions. -For example, the following authorization managers definition enables the "basic" implementation from `druid-basic-security`: +For example, the following authorizers definition enables the "basic" implementation from `druid-basic-security`: ```json ["basic"] ``` -### Default Authorization Manager -The default AuthorizationManager with type name "default" rejects all requests. +### Default Authorizer +The default Authorizer with type name "default" rejects all requests. -### No-op Authorization Manager -The no-op AuthorizationManager with type name "noop" accepts all requests. +### No-op Authorizer +The no-op Authorizer with type name "noop" accepts all requests. ## Namespaces -Authenticator and AuthorizationManager implementations are linked through a namespace string. Authenticators tag an authenticated request with a namespace, which is used to route the authenticated request to the AuthorizationManager implementation that registered itself with a matching namespace. +Authenticator and Authorizer implementations are linked through a namespace string. Authenticators tag an authenticated request with a namespace, which is used to route the authenticated request to the Authorizer implementation that registered itself with a matching namespace. -This is to support cases where an AuthorizationManager implementation is only intended to authorize requests from a specific authenticator (an implementation may have assumptions about the user name format, for example). +This is to support cases where an Authorizer implementation is only intended to authorize requests from a specific authenticator (an implementation may have assumptions about the user name format, for example). -The details of namespace configuration are left for implementors of Authenticator and AuthorizationManager to decide. \ No newline at end of file +The details of namespace configuration are left for implementors of Authenticator and Authorizer to decide. \ No newline at end of file diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 109537601337..8b3845b2db45 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -76,7 +76,7 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -140,7 +140,7 @@ public enum Status private final KafkaTuningConfig tuningConfig; private final KafkaIOConfig ioConfig; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; private final Optional chatHandlerProvider; private final Map endOffsets = new ConcurrentHashMap<>(); @@ -207,7 +207,7 @@ public KafkaIndexTask( @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, @JacksonInject AuthConfig authConfig, - @JacksonInject AuthorizationManagerMapper authorizationManagerMapper + @JacksonInject AuthorizerMapper authorizerMapper ) { super( @@ -224,7 +224,7 @@ public KafkaIndexTask( this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); } @@ -634,7 +634,7 @@ private Access authorizationCheck(final HttpServletRequest req, Action action) action ); - return AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizationManagerMapper); + return AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index cf97bbac86f7..f9d39573b2e0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -56,7 +56,7 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -100,7 +100,7 @@ public class OverlordResource private final JacksonConfigManager configManager; private final AuditManager auditManager; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; private AtomicReference workerConfigRef = null; @@ -112,7 +112,7 @@ public OverlordResource( JacksonConfigManager configManager, AuditManager auditManager, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) throws Exception { this.taskMaster = taskMaster; @@ -121,7 +121,7 @@ public OverlordResource( this.configManager = configManager; this.auditManager = auditManager; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } @POST @@ -143,7 +143,7 @@ public Response taskPost( Access authResult = AuthorizationUtils.authorizeResourceAction( req, resourceAction, - authorizationManagerMapper + authorizerMapper ); if (!authResult.isAllowed()) { @@ -389,7 +389,7 @@ public ResourceAction apply(Task input) req, allActiveTasks, raGenerator, - authorizationManagerMapper + authorizerMapper ); } else { @@ -511,7 +511,7 @@ public ResourceAction apply(TaskStatus input) req, taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(), raGenerator, - authorizationManagerMapper + authorizerMapper ); } else { recentlyFinishedTasks = taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(); @@ -697,7 +697,7 @@ public ResourceAction apply(TaskRunnerWorkItem input) req, collectionToFilter, raGenerator, - authorizationManagerMapper + authorizerMapper ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java index f1952b4bcf3f..2fec23da3fb2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java @@ -34,7 +34,7 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.ResourceAction; @@ -50,11 +50,11 @@ public class SupervisorResourceFilter extends AbstractResourceFilter @Inject public SupervisorResourceFilter( AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper, + AuthorizerMapper authorizerMapper, SupervisorManager supervisorManager ) { - super(authConfig, authorizationManagerMapper); + super(authConfig, authorizerMapper); this.supervisorManager = supervisorManager; } @@ -103,7 +103,7 @@ public boolean apply(PathSegment input) getReq(), spec.getDataSources(), resourceActionFunction, - getAuthorizationManagerMapper() + getAuthorizerMapper() ); if (!authResult.isAllowed()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java index c7264b259b54..ac41eb0cc069 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java @@ -32,7 +32,7 @@ import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -57,10 +57,10 @@ public class TaskResourceFilter extends AbstractResourceFilter public TaskResourceFilter( TaskStorageQueryAdapter taskStorageQueryAdapter, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizationManagerMapper); + super(authConfig, authorizerMapper); this.taskStorageQueryAdapter = taskStorageQueryAdapter; } @@ -103,7 +103,7 @@ public boolean apply(PathSegment input) final Access authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, - getAuthorizationManagerMapper() + getAuthorizerMapper() ); if (!authResult.isAllowed()) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java index 2e5baa673832..35bc4bcb2d97 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -33,7 +33,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import javax.servlet.http.HttpServletRequest; @@ -59,18 +59,18 @@ public class SupervisorResource { private final TaskMaster taskMaster; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public SupervisorResource( TaskMaster taskMaster, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { this.taskMaster = taskMaster; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } @POST @@ -94,7 +94,7 @@ public Response apply(SupervisorManager manager) req, spec.getDataSources(), AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); if (!authResult.isAllowed()) { @@ -128,7 +128,7 @@ public Response apply(final SupervisorManager manager) req, supervisorSpecOptional.get().getDataSources(), AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); if (accessResult.isAllowed()) { @@ -247,7 +247,7 @@ public boolean apply(String id) req, supervisorSpecOptional.get().getDataSources(), AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); return accessResult.isAllowed(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index 0bfb897b3ae9..d35346e93c10 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -34,7 +34,7 @@ import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import java.util.List; import java.util.concurrent.TimeUnit; @@ -75,7 +75,7 @@ public int columnCacheSizeBytes() .addValue(ObjectMapper.class, jsonMapper) .addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider()) .addValue(AuthConfig.class, new AuthConfig()) - .addValue(AuthorizationManagerMapper.class, null) + .addValue(AuthorizerMapper.class, null) ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 3400d4c8786d..beca5c0028d7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -39,8 +39,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; import org.easymock.EasyMock; import org.junit.After; @@ -73,11 +73,11 @@ public void setUp() throws Exception Optional.of(taskRunner) ).anyTimes(); - AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new AuthorizationManager() + return new Authorizer() { @Override public Access authorize(String identity, Resource resource, Action action) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java index cb60e4ec52fb..2f3cdb0d7c8b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java @@ -35,7 +35,7 @@ import io.druid.indexing.worker.http.WorkerResource; import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.http.security.ResourceFilterTestHelper; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -60,15 +60,15 @@ public static Collection data() Iterables.concat( getRequestPaths(OverlordResource.class, ImmutableList.>of( TaskStorageQueryAdapter.class, - AuthorizationManagerMapper.class + AuthorizerMapper.class ) ), getRequestPaths(WorkerResource.class, ImmutableList.>of( - AuthorizationManagerMapper.class + AuthorizerMapper.class )), getRequestPaths(SupervisorResource.class, ImmutableList.>of( SupervisorManager.class, - AuthorizationManagerMapper.class + AuthorizerMapper.class ) ) ) @@ -150,7 +150,7 @@ public void testResourcesFilteringAccess() // As request object is a strict mock the ordering of expected calls matters // therefore adding the expectation below again as getEntity is called before getMethod EasyMock.expect(request.getMethod()).andReturn(requestMethod).anyTimes(); - EasyMock.replay(req, request, authorizationManagerMapper); + EasyMock.replay(req, request, authorizerMapper); resourceFilter.getRequestFilter().filter(request); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); } @@ -160,7 +160,7 @@ public void testDatasourcesResourcesFilteringNoAccess() { setUpMockExpectations(requestPath, false, requestMethod); EasyMock.expect(request.getEntity(Task.class)).andReturn(noopTask).anyTimes(); - EasyMock.replay(req, request, authorizationManagerMapper); + EasyMock.replay(req, request, authorizerMapper); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); try { resourceFilter.getRequestFilter().filter(request); @@ -176,14 +176,14 @@ public void testDatasourcesResourcesFilteringBadPath() { final String badRequestPath = requestPath.replaceAll("\\w+", "droid"); EasyMock.expect(request.getPath()).andReturn(badRequestPath).anyTimes(); - EasyMock.replay(req, request, authorizationManagerMapper); + EasyMock.replay(req, request, authorizerMapper); Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath)); } @After public void tearDown() { - EasyMock.verify(req, request, authorizationManagerMapper); + EasyMock.verify(req, request, authorizerMapper); if (tsqa != null) { EasyMock.verify(tsqa); } diff --git a/server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java b/server/src/main/java/io/druid/guice/security/AuthorizerModule.java similarity index 71% rename from server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java rename to server/src/main/java/io/druid/guice/security/AuthorizerModule.java index 3fe687fe45f3..e711bda1b19c 100644 --- a/server/src/main/java/io/druid/guice/security/AuthorizationManagerModule.java +++ b/server/src/main/java/io/druid/guice/security/AuthorizerModule.java @@ -28,26 +28,26 @@ import io.druid.guice.LazySingleton; import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.NoopAuthorizationManager; +import io.druid.server.security.Authorizer; +import io.druid.server.security.NoopAuthorizer; -public class AuthorizationManagerModule implements Module +public class AuthorizerModule implements Module { @Override public void configure(Binder binder) { - final MapBinder authorizationManagerMapBinder = PolyBind.optionBinder( + final MapBinder authorizerMapBinder = PolyBind.optionBinder( binder, - Key.get(AuthorizationManager.class) + Key.get(Authorizer.class) ); - authorizationManagerMapBinder.addBinding("noop").to(NoopAuthorizationManager.class).in(LazySingleton.class); + authorizerMapBinder.addBinding("noop").to(NoopAuthorizer.class).in(LazySingleton.class); } @Provides @ManageLifecycle @Named("noop") - public AuthorizationManager getAuthorizationManager() + public Authorizer getAuthorizer() { - return new NoopAuthorizationManager(); + return new NoopAuthorizer(); } } diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index f1f526ebfe48..3111d2eb6986 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -57,13 +57,13 @@ import io.druid.guice.annotations.Smile; import io.druid.guice.http.HttpClientModule; import io.druid.guice.security.AuthenticatorModule; -import io.druid.guice.security.AuthorizationManagerModule; +import io.druid.guice.security.AuthorizerModule; import io.druid.guice.security.DruidAuthModule; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; import io.druid.server.initialization.AuthenticatorHttpClientWrapperModule; -import io.druid.server.initialization.AuthorizationManagerMapperModule; +import io.druid.server.initialization.AuthorizerMapperModule; import io.druid.server.initialization.EmitterModule; import io.druid.server.initialization.jetty.JettyServerModule; import io.druid.server.metrics.MetricsModule; @@ -350,8 +350,8 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new DruidAuthModule(), new AuthenticatorModule(), new AuthenticatorHttpClientWrapperModule(), - new AuthorizationManagerModule(), - new AuthorizationManagerMapperModule(), + new AuthorizerModule(), + new AuthorizerMapperModule(), new LifecycleModule(), EmitterModule.class, HttpClientModule.global(), diff --git a/server/src/main/java/io/druid/server/BrokerQueryResource.java b/server/src/main/java/io/druid/server/BrokerQueryResource.java index cea6b6a94799..a55ef4bd7511 100644 --- a/server/src/main/java/io/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/io/druid/server/BrokerQueryResource.java @@ -31,7 +31,7 @@ import io.druid.query.Query; import io.druid.server.http.security.StateResourceFilter; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -60,7 +60,7 @@ public BrokerQueryResource( @Smile ObjectMapper smileMapper, QueryManager queryManager, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper, + AuthorizerMapper authorizerMapper, GenericQueryMetricsFactory queryMetricsFactory, TimelineServerView brokerServerView ) @@ -71,7 +71,7 @@ public BrokerQueryResource( smileMapper, queryManager, authConfig, - authorizationManagerMapper, + authorizerMapper, queryMetricsFactory ); this.brokerServerView = brokerServerView; diff --git a/server/src/main/java/io/druid/server/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java index da29ff954049..084a4aaaf679 100644 --- a/server/src/main/java/io/druid/server/ClientInfoResource.java +++ b/server/src/main/java/io/druid/server/ClientInfoResource.java @@ -39,7 +39,7 @@ import io.druid.query.metadata.SegmentMetadataQueryConfig; import io.druid.server.http.security.DatasourceResourceFilter; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; @@ -79,7 +79,7 @@ public class ClientInfoResource private TimelineServerView timelineServerView; private SegmentMetadataQueryConfig segmentMetadataQueryConfig; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public ClientInfoResource( @@ -87,7 +87,7 @@ public ClientInfoResource( TimelineServerView timelineServerView, SegmentMetadataQueryConfig segmentMetadataQueryConfig, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; @@ -95,7 +95,7 @@ public ClientInfoResource( this.segmentMetadataQueryConfig = (segmentMetadataQueryConfig == null) ? new SegmentMetadataQueryConfig() : segmentMetadataQueryConfig; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } private Map> getSegmentsForDatasources() @@ -122,7 +122,7 @@ public Iterable getDataSources(@Context final HttpServletRequest request request, getSegmentsForDatasources().keySet(), AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); } else { return getSegmentsForDatasources().keySet(); diff --git a/server/src/main/java/io/druid/server/QueryLifecycle.java b/server/src/main/java/io/druid/server/QueryLifecycle.java index 510f61c616ae..c2262c87de5b 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycle.java +++ b/server/src/main/java/io/druid/server/QueryLifecycle.java @@ -40,7 +40,7 @@ import io.druid.server.log.RequestLogger; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import org.joda.time.DateTime; @@ -75,7 +75,7 @@ public class QueryLifecycle private final RequestLogger requestLogger; private final ServerConfig serverConfig; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; private final long startMs; private final long startNs; @@ -91,7 +91,7 @@ public QueryLifecycle( final RequestLogger requestLogger, final ServerConfig serverConfig, final AuthConfig authConfig, - final AuthorizationManagerMapper authorizationManagerMapper, + final AuthorizerMapper authorizerMapper, final long startMs, final long startNs ) @@ -103,7 +103,7 @@ public QueryLifecycle( this.requestLogger = requestLogger; this.serverConfig = serverConfig; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; this.startMs = startMs; this.startNs = startNs; } @@ -217,7 +217,7 @@ public Access authorize( req, queryPlus.getQuery().getDataSource().getNames(), AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); } else { authResult = AuthorizationUtils.authorizeAllResourceActions( @@ -225,7 +225,7 @@ public Access authorize( AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, token, namespace, - authorizationManagerMapper + authorizerMapper ); } diff --git a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java index 4318ddcd6280..eb698e838fc0 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java +++ b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java @@ -28,7 +28,7 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; @LazySingleton public class QueryLifecycleFactory @@ -40,7 +40,7 @@ public class QueryLifecycleFactory private final RequestLogger requestLogger; private final ServerConfig serverConfig; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public QueryLifecycleFactory( @@ -51,7 +51,7 @@ public QueryLifecycleFactory( final RequestLogger requestLogger, final ServerConfig serverConfig, final AuthConfig authConfig, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { this.warehouse = warehouse; @@ -61,7 +61,7 @@ public QueryLifecycleFactory( this.requestLogger = requestLogger; this.serverConfig = serverConfig; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } public QueryLifecycle factorize() @@ -74,7 +74,7 @@ public QueryLifecycle factorize() requestLogger, serverConfig, authConfig, - authorizationManagerMapper, + authorizerMapper, System.currentTimeMillis(), System.nanoTime() ); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index ebcbaaf24d51..7bf1271dec09 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -45,7 +45,7 @@ import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import org.joda.time.DateTime; @@ -91,7 +91,7 @@ public class QueryResource implements QueryCountStatsProvider protected final ObjectMapper serializeDateTimeAsLongSmileMapper; protected final QueryManager queryManager; protected final AuthConfig authConfig; - protected final AuthorizationManagerMapper authorizationManagerMapper; + protected final AuthorizerMapper authorizerMapper; private final GenericQueryMetricsFactory queryMetricsFactory; private final AtomicLong successfulQueryCount = new AtomicLong(); @@ -105,7 +105,7 @@ public QueryResource( @Smile ObjectMapper smileMapper, QueryManager queryManager, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper, + AuthorizerMapper authorizerMapper, GenericQueryMetricsFactory queryMetricsFactory ) { @@ -116,7 +116,7 @@ public QueryResource( this.serializeDateTimeAsLongSmileMapper = serializeDataTimeAsLong(smileMapper); this.queryManager = queryManager; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; this.queryMetricsFactory = queryMetricsFactory; } @@ -139,7 +139,7 @@ public Response getServer(@PathParam("id") String queryId, @Context final HttpSe req, datasources, AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); if (!authResult.isAllowed()) { diff --git a/server/src/main/java/io/druid/server/http/DatasourcesResource.java b/server/src/main/java/io/druid/server/http/DatasourcesResource.java index 66798e1ee898..92be2e73264c 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -42,7 +42,7 @@ import io.druid.query.TableDataSource; import io.druid.server.http.security.DatasourceResourceFilter; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -80,7 +80,7 @@ public class DatasourcesResource private final MetadataSegmentManager databaseSegmentManager; private final IndexingServiceClient indexingServiceClient; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public DatasourcesResource( @@ -88,14 +88,14 @@ public DatasourcesResource( MetadataSegmentManager databaseSegmentManager, @Nullable IndexingServiceClient indexingServiceClient, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; this.databaseSegmentManager = databaseSegmentManager; this.indexingServiceClient = indexingServiceClient; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } @GET @@ -110,7 +110,7 @@ public Response getQueryableDataSources( final Set datasources = authConfig.isEnabled() ? InventoryViewUtils.getSecuredDataSources( serverInventoryView, - authorizationManagerMapper, + authorizerMapper, (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) ) : diff --git a/server/src/main/java/io/druid/server/http/IntervalsResource.java b/server/src/main/java/io/druid/server/http/IntervalsResource.java index b7facd67a8d9..32f08ec81978 100644 --- a/server/src/main/java/io/druid/server/http/IntervalsResource.java +++ b/server/src/main/java/io/druid/server/http/IntervalsResource.java @@ -27,7 +27,7 @@ import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.guava.Comparators; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -51,18 +51,18 @@ public class IntervalsResource { private final InventoryView serverInventoryView; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public IntervalsResource( InventoryView serverInventoryView, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } @GET @@ -73,7 +73,7 @@ public Response getIntervals(@Context final HttpServletRequest req) final Set datasources = authConfig.isEnabled() ? InventoryViewUtils.getSecuredDataSources( serverInventoryView, - authorizationManagerMapper, + authorizerMapper, (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) ) : @@ -108,7 +108,7 @@ public Response getSpecificIntervals( final Set datasources = authConfig.isEnabled() ? InventoryViewUtils.getSecuredDataSources( serverInventoryView, - authorizationManagerMapper, + authorizerMapper, (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) ) : diff --git a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java index be410a51173c..50e0ec52e6e4 100644 --- a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java +++ b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java @@ -33,8 +33,8 @@ import io.druid.java.util.common.Pair; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; import io.druid.server.security.ResourceType; @@ -81,18 +81,18 @@ public Iterable apply(DruidServer input) public static Set getSecuredDataSources( InventoryView inventoryView, - final AuthorizationManagerMapper authorizationManagerMapper, + final AuthorizerMapper authorizerMapper, final String identity, final String namespace ) { - if (authorizationManagerMapper == null) { + if (authorizerMapper == null) { throw new ISE("No authorization mapper found"); } - final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); - if (authorizationManager == null) { - throw new ISE("Invalid to call a secured method with null AuthorizationManager!!"); + final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + if (authorizer == null) { + throw new ISE("Invalid to call a secured method with null Authorizer!!"); } else { final Map, Access> resourceAccessMap = new HashMap<>(); return ImmutableSet.copyOf( @@ -109,7 +109,7 @@ public boolean apply(DruidDataSource input) if (resourceAccessMap.containsKey(key)) { return resourceAccessMap.get(key).isAllowed(); } else { - Access access = authorizationManager.authorize(identity, key.lhs, key.rhs); + Access access = authorizer.authorize(identity, key.lhs, key.rhs); resourceAccessMap.put(key, access); return access.isAllowed(); } diff --git a/server/src/main/java/io/druid/server/http/MetadataResource.java b/server/src/main/java/io/druid/server/http/MetadataResource.java index f6034fec1e20..6f6b007dd91e 100644 --- a/server/src/main/java/io/druid/server/http/MetadataResource.java +++ b/server/src/main/java/io/druid/server/http/MetadataResource.java @@ -31,7 +31,7 @@ import io.druid.metadata.MetadataSegmentManager; import io.druid.server.http.security.DatasourceResourceFilter; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -58,20 +58,20 @@ public class MetadataResource private final MetadataSegmentManager metadataSegmentManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public MetadataResource( MetadataSegmentManager metadataSegmentManager, IndexerMetadataStorageCoordinator metadataStorageCoordinator, AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { this.metadataSegmentManager = metadataSegmentManager; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } @GET @@ -109,7 +109,7 @@ public String apply(DruidDataSource input) req, dataSourceNamesPreAuth, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); dataSourceNamesPostAuth = Sets.newTreeSet(datasourceNamesList); diff --git a/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java b/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java index 19f372b40d4e..09e2a955c0b6 100644 --- a/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java @@ -26,7 +26,7 @@ import com.sun.jersey.spi.container.ResourceFilter; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Context; @@ -38,16 +38,16 @@ public abstract class AbstractResourceFilter implements ResourceFilter, Containe private HttpServletRequest req; private final AuthConfig authConfig; - private AuthorizationManagerMapper authorizationManagerMapper; + private AuthorizerMapper authorizerMapper; @Inject public AbstractResourceFilter( AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } @Override @@ -72,14 +72,14 @@ public AuthConfig getAuthConfig() return authConfig; } - public AuthorizationManagerMapper getAuthorizationManagerMapper() + public AuthorizerMapper getAuthorizerMapper() { - return authorizationManagerMapper; + return authorizerMapper; } - public void setAuthorizationManagerMapper(AuthorizationManagerMapper authorizationManagerMapper) + public void setAuthorizerMapper(AuthorizerMapper authorizerMapper) { - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } public AbstractResourceFilter setReq(HttpServletRequest req) diff --git a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java index 0baaedcc6181..3091e443ae45 100644 --- a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java @@ -24,7 +24,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -47,10 +47,10 @@ public class ConfigResourceFilter extends AbstractResourceFilter @Inject public ConfigResourceFilter( AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizationManagerMapper); + super(authConfig, authorizerMapper); } @Override @@ -65,7 +65,7 @@ public ContainerRequest filter(ContainerRequest request) final Access authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, - getAuthorizationManagerMapper() + getAuthorizerMapper() ); if (!authResult.isAllowed()) { diff --git a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java index f28390440657..46809464f1b3 100644 --- a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java @@ -28,7 +28,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -51,10 +51,10 @@ public class DatasourceResourceFilter extends AbstractResourceFilter @Inject public DatasourceResourceFilter( AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizationManagerMapper); + super(authConfig, authorizerMapper); } @Override @@ -69,7 +69,7 @@ public ContainerRequest filter(ContainerRequest request) final Access authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, - getAuthorizationManagerMapper() + getAuthorizerMapper() ); if (!authResult.isAllowed()) { diff --git a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java index 221f62d5433e..3cbe5cde93e9 100644 --- a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java @@ -28,7 +28,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -51,10 +51,10 @@ public class RulesResourceFilter extends AbstractResourceFilter @Inject public RulesResourceFilter( AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizationManagerMapper); + super(authConfig, authorizerMapper); } @Override @@ -85,7 +85,7 @@ public boolean apply(PathSegment input) final Access authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, - getAuthorizationManagerMapper() + getAuthorizerMapper() ); if (!authResult.isAllowed()) { diff --git a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java index 7fafc9877f33..e2074793f39e 100644 --- a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java @@ -24,7 +24,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -53,10 +53,10 @@ public class StateResourceFilter extends AbstractResourceFilter @Inject public StateResourceFilter( AuthConfig authConfig, - AuthorizationManagerMapper authorizationManagerMapper + AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizationManagerMapper); + super(authConfig, authorizerMapper); } @Override @@ -71,7 +71,7 @@ public ContainerRequest filter(ContainerRequest request) final Access authResult = AuthorizationUtils.authorizeResourceAction( getReq(), resourceAction, - getAuthorizationManagerMapper() + getAuthorizerMapper() ); if (!authResult.isAllowed()) { diff --git a/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java similarity index 55% rename from server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java rename to server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java index b3714febbb7f..04bf800b14f0 100644 --- a/server/src/main/java/io/druid/server/initialization/AuthorizationManagerMapperModule.java +++ b/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java @@ -32,26 +32,26 @@ import io.druid.initialization.DruidModule; import io.druid.java.util.common.logger.Logger; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; -import io.druid.server.security.DefaultAuthorizationManager; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.DefaultAuthorizer; import java.util.Collections; import java.util.List; import java.util.Map; -public class AuthorizationManagerMapperModule implements DruidModule +public class AuthorizerMapperModule implements DruidModule { - private static Logger log = new Logger(AuthorizationManagerMapperModule.class); + private static Logger log = new Logger(AuthorizerMapperModule.class); @Override public void configure(Binder binder) { - binder.bind(AuthorizationManagerMapper.class) - .toProvider(new AuthorizationManagerMapperProvider()) + binder.bind(AuthorizerMapper.class) + .toProvider(new AuthorizerMapperProvider()) .in(LazySingleton.class); - LifecycleModule.register(binder, AuthorizationManagerMapper.class); + LifecycleModule.register(binder, AuthorizerMapper.class); } @SuppressWarnings("unchecked") @@ -61,7 +61,7 @@ public List getJacksonModules() return Collections.EMPTY_LIST; } - private static class AuthorizationManagerMapperProvider implements Provider + private static class AuthorizerMapperProvider implements Provider { private AuthConfig authConfig; private Injector injector; @@ -74,33 +74,33 @@ public void inject(Injector injector) } @Override - public AuthorizationManagerMapper get() + public AuthorizerMapper get() { - Map authorizationManagerMap = Maps.newHashMap(); + Map authorizerMap = Maps.newHashMap(); - List authorizationManagers = authConfig.getAuthorizationManagers(); + List authorizers = authConfig.getAuthorizers(); - // If user didn't configure any AuthorizationManagers, use the default which rejects all requests. - if (authorizationManagers == null || authorizationManagers.isEmpty()) { - return new AuthorizationManagerMapper(null) { + // If user didn't configure any Authorizers, use the default which rejects all requests. + if (authorizers == null || authorizers.isEmpty()) { + return new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new DefaultAuthorizationManager(); + return new DefaultAuthorizer(); } }; } - for (String authorizationManagerName : authorizationManagers) { - AuthorizationManager authorizationManager = injector.getInstance(Key.get( - AuthorizationManager.class, - Names.named(authorizationManagerName) + for (String authorizerName : authorizers) { + Authorizer authorizer = injector.getInstance(Key.get( + Authorizer.class, + Names.named(authorizerName) )); - authorizationManagerMap.put(authorizationManager.getNamespace(), authorizationManager); + authorizerMap.put(authorizer.getNamespace(), authorizer); } - return new AuthorizationManagerMapper(authorizationManagerMap); + return new AuthorizerMapper(authorizerMap); } } } diff --git a/server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java b/server/src/main/java/io/druid/server/initialization/NoopAuthorizerModule.java similarity index 81% rename from server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java rename to server/src/main/java/io/druid/server/initialization/NoopAuthorizerModule.java index 254c92ce5e4f..244ce4ee4556 100644 --- a/server/src/main/java/io/druid/server/initialization/NoopAuthorizationManagerModule.java +++ b/server/src/main/java/io/druid/server/initialization/NoopAuthorizerModule.java @@ -24,10 +24,10 @@ import com.google.inject.Provides; import com.google.inject.name.Named; import io.druid.guice.ManageLifecycle; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.NoopAuthorizationManager; +import io.druid.server.security.Authorizer; +import io.druid.server.security.NoopAuthorizer; -public class NoopAuthorizationManagerModule implements Module +public class NoopAuthorizerModule implements Module { public static final String TYPE = "noop"; @@ -39,8 +39,8 @@ public void configure(Binder binder) @Provides @ManageLifecycle @Named(TYPE) - public AuthorizationManager makeAuthorizationManager() + public Authorizer makeAuthorizer() { - return new NoopAuthorizationManager(); + return new NoopAuthorizer(); } } diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index 57c8f0661e10..424f67ee1bd3 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -39,7 +39,7 @@ public class AuthConfig /** * HTTP attribute that indicates the namespace for a request. Set by Authenticator implementations when - * they successfully authenticate a request. The AuthorizationManager with a matching namespace will be used to + * they successfully authenticate a request. The Authorizer with a matching namespace will be used to * authorize the request. */ public static final String DRUID_AUTH_NAMESPACE = "Druid-Auth-Namespace"; @@ -54,13 +54,13 @@ public AuthConfig( @JsonProperty("enabled") boolean enabled, @JsonProperty("authenticatorChain") List authenticationChain, @JsonProperty("internalAuthenticator") String internalAuthenticator, - @JsonProperty("authorizationManagers") List authorizationManagers + @JsonProperty("authorizers") List authorizers ) { this.enabled = enabled; this.authenticatorChain = authenticationChain; this.internalAuthenticator = internalAuthenticator; - this.authorizationManagers = authorizationManagers; + this.authorizers = authorizers; } @JsonProperty @@ -73,7 +73,7 @@ public AuthConfig( private final String internalAuthenticator; @JsonProperty - private List authorizationManagers; + private List authorizers; public boolean isEnabled() { @@ -90,9 +90,9 @@ public String getInternalAuthenticator() return internalAuthenticator; } - public List getAuthorizationManagers() + public List getAuthorizers() { - return authorizationManagers; + return authorizers; } @Override @@ -102,7 +102,7 @@ public String toString() "enabled=" + enabled + ", authenticatorChain='" + authenticatorChain + '\'' + ", internalAuthenticator='" + internalAuthenticator + '\'' + - ", authorizationManagers='" + authorizationManagers + '\'' + + ", authorizers='" + authorizers + '\'' + '}'; } @@ -131,9 +131,9 @@ public boolean equals(Object o) : that.getInternalAuthenticator() != null) { return false; } - return getAuthorizationManagers() != null - ? getAuthorizationManagers().equals(that.getAuthorizationManagers()) - : that.getAuthorizationManagers() == null; + return getAuthorizers() != null + ? getAuthorizers().equals(that.getAuthorizers()) + : that.getAuthorizers() == null; } @@ -143,7 +143,7 @@ public int hashCode() int result = (isEnabled() ? 1 : 0); result = 31 * result + (getAuthenticatorChain() != null ? getAuthenticatorChain().hashCode() : 0); result = 31 * result + (getInternalAuthenticator() != null ? getInternalAuthenticator().hashCode() : 0); - result = 31 * result + (getAuthorizationManagers() != null ? getAuthorizationManagers().hashCode() : 0); + result = 31 * result + (getAuthorizers() != null ? getAuthorizers().hashCode() : 0); return result; } diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index 0b62ea2ec2e4..344f85df1221 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -49,7 +49,7 @@ public interface Authenticator extends ServletFilterHolder /** * @return The namespace associated with this Authenticator. This will be used for choosing the correct - * AuthorizationManager for authorizing requests that have been authenticated by this Authenticator. + * Authorizer for authorizing requests that have been authenticated by this Authenticator. */ public String getNamespace(); diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index 9f6e4fff3a02..f433a64250b2 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -50,19 +50,19 @@ public class AuthorizationUtils * * @param request HTTP request to be authorized * @param resourceAction A resource identifier and the action to be taken the resource. - * @param authorizationManagerMapper The singleton AuthorizationManagerMapper instance - * @return ACCESS_OK or the failed Access object returned by the AuthorizationManager that checked the request. + * @param authorizerMapper The singleton AuthorizerMapper instance + * @return ACCESS_OK or the failed Access object returned by the Authorizer that checked the request. */ public static Access authorizeResourceAction( final HttpServletRequest request, final ResourceAction resourceAction, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { return authorizeAllResourceActions( request, Lists.newArrayList(resourceAction), - authorizationManagerMapper + authorizerMapper ); } @@ -84,7 +84,7 @@ public static Access authorizeResourceAction( public static Access authorizeAllResourceActions( final HttpServletRequest request, final List resourceActions, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); @@ -97,9 +97,9 @@ public static Access authorizeAllResourceActions( throw new ISE("Null namespace."); } - final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); - if (authorizationManager == null) { - throw new ISE("No authorization manager found for namespace: [%s].", namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + if (authorizer == null) { + throw new ISE("No authorizer found for namespace: [%s].", namespace); } // this method returns on first failure, so only successful Access results are kept in the cache @@ -109,7 +109,7 @@ public static Access authorizeAllResourceActions( if (resultCache.contains(resourceAction)) { continue; } - final Access access = authorizationManager.authorize( + final Access access = authorizer.authorize( identity, resourceAction.getResource(), resourceAction.getAction() @@ -137,7 +137,7 @@ public static Access authorizeAllResourceActions( * * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. * - * If this attribute is already set when this fImmutableList.>of(SupervisorManager.class, AuthorizationManager.class)unction is called, an exception is thrown. + * If this attribute is already set when this fImmutableList.>of(SupervisorManager.class, Authorizer.class)unction is called, an exception is thrown. * * @param request HTTP request to be generated * @param resources List of resources @@ -149,7 +149,7 @@ public static Access authorizeAllResourceActions( final HttpServletRequest request, final Collection resources, final Function raGenerator, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); @@ -162,9 +162,9 @@ public static Access authorizeAllResourceActions( throw new ISE("Null namespace."); } - final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); - if (authorizationManager == null) { - throw new ISE("No authorization manager found for namespace: [%s].", namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + if (authorizer == null) { + throw new ISE("No authorizer found for namespace: [%s].", namespace); } // this method returns on first failure, so only successful Access results are kept in the cache @@ -175,7 +175,7 @@ public static Access authorizeAllResourceActions( if (resultCache.contains(resourceAction)) { continue; } - final Access access = authorizationManager.authorize( + final Access access = authorizer.authorize( identity, resourceAction.getResource(), resourceAction.getAction() @@ -211,15 +211,15 @@ public static Access authorizeAllResourceActions( final Function raGenerator, final String user, final String namespace, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { if (user == null || namespace == null) { throw new ISE("null user or namespace"); } - final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); - if (authorizationManager == null) { - throw new ISE("No authorization manager found for namespace: [%s].", namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + if (authorizer == null) { + throw new ISE("No authorizer found for namespace: [%s].", namespace); } // this method returns on first failure, so only successful Access results are kept in the cache @@ -230,7 +230,7 @@ public static Access authorizeAllResourceActions( if (resultCache.contains(resourceAction)) { continue; } - final Access access = authorizationManager.authorize( + final Access access = authorizer.authorize( user, resourceAction.getResource(), resourceAction.getAction() @@ -262,7 +262,7 @@ public static List filterAuthorizedResources( final HttpServletRequest request, final Collection resources, final Function resourceActionGenerator, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); @@ -275,9 +275,9 @@ public static List filterAuthorizedResources( throw new ISE("Null namespace."); } - final AuthorizationManager authorizationManager = authorizationManagerMapper.getAuthorizationManager(namespace); - if (authorizationManager == null) { - throw new ISE("No authorization manager found for namespace: [%s].", namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + if (authorizer == null) { + throw new ISE("No authorizer found for namespace: [%s].", namespace); } int initialSize = resources.size(); @@ -287,7 +287,7 @@ public static List filterAuthorizedResources( final ResourceAction resourceAction = resourceActionGenerator.apply(resource); Access access = resultCache.get(resourceAction); if (access == null) { - access = authorizationManager.authorize( + access = authorizer.authorize( identity, resourceAction.getResource(), resourceAction.getAction() diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManager.java b/server/src/main/java/io/druid/server/security/Authorizer.java similarity index 79% rename from server/src/main/java/io/druid/server/security/AuthorizationManager.java rename to server/src/main/java/io/druid/server/security/Authorizer.java index d159168d7b47..359bee70cb4e 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationManager.java +++ b/server/src/main/java/io/druid/server/security/Authorizer.java @@ -22,23 +22,23 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultAuthorizationManager.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultAuthorizer.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultAuthorizationManager.class), - @JsonSubTypes.Type(name = "noop", value = NoopAuthorizationManager.class) + @JsonSubTypes.Type(name = "default", value = DefaultAuthorizer.class), + @JsonSubTypes.Type(name = "noop", value = NoopAuthorizer.class) }) /** - * An AuthorizationManager is responsible for performing authorization checks for resource accesses. + * An Authorizer is responsible for performing authorization checks for resource accesses. * - * A single instance of each AuthorizationManager implementation will be created per node. + * A single instance of each Authorizer implementation will be created per node. * Security-sensitive endpoints will need to extract the identity string contained in the request's Druid-Auth-Token * attribute, previously set by an Authenticator. Each endpoint will pass this identity String to the - * AuthorizationManager's authorize() method along with any Resource/Action pairs created for the request being + * Authorizer's authorize() method along with any Resource/Action pairs created for the request being * handled. The endpoint can use these checks to filter out resources or deny the request as needed. * After a request is authorized, a new attribute, "Druid-Auth-Token-Checked", should be set in the * request header with the result of the authorization decision. */ -public interface AuthorizationManager +public interface Authorizer { /** * Check if the entity represented by `identity` in `namespace` is authorized to perform `action` on `resource`. @@ -53,7 +53,7 @@ public interface AuthorizationManager Access authorize(String identity, Resource resource, Action action); /** - * @return The namespace associated with this AuthorizationManager. Authenticator implementations will + * @return The namespace associated with this Authorizer. Authenticator implementations will * put the namespace in request headers. */ String getNamespace(); diff --git a/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java similarity index 63% rename from server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java rename to server/src/main/java/io/druid/server/security/AuthorizerMapper.java index 7caa74287b87..e6caf0a85b09 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationManagerMapper.java +++ b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java @@ -25,35 +25,35 @@ import java.util.Map; @ManageLifecycle -public class AuthorizationManagerMapper +public class AuthorizerMapper { - private Map authorizationManagerMap; + private Map authorizerMap; - public AuthorizationManagerMapper( - Map authorizationManagerMap + public AuthorizerMapper( + Map authorizerMap ) { - this.authorizationManagerMap = authorizationManagerMap; + this.authorizerMap = authorizerMap; } - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return authorizationManagerMap.get(namespace); + return authorizerMap.get(namespace); } @LifecycleStart public void start() { - for (AuthorizationManager authorizationManager : authorizationManagerMap.values()) { - authorizationManager.start(); + for (Authorizer authorizer : authorizerMap.values()) { + authorizer.start(); } } @LifecycleStart public void stop() { - for (AuthorizationManager authorizationManager : authorizationManagerMap.values()) { - authorizationManager.stop(); + for (Authorizer authorizer : authorizerMap.values()) { + authorizer.stop(); } } } diff --git a/server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java b/server/src/main/java/io/druid/server/security/DefaultAuthorizer.java similarity index 91% rename from server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java rename to server/src/main/java/io/druid/server/security/DefaultAuthorizer.java index 7169e121d2a1..97bf9e6e4930 100644 --- a/server/src/main/java/io/druid/server/security/DefaultAuthorizationManager.java +++ b/server/src/main/java/io/druid/server/security/DefaultAuthorizer.java @@ -19,12 +19,12 @@ package io.druid.server.security; -public class DefaultAuthorizationManager implements AuthorizationManager +public class DefaultAuthorizer implements Authorizer { @Override public Access authorize(String identity, Resource resource, Action action) { - return new Access(false, "Please configure a non-default Authorization Manager."); + return new Access(false, "Please configure a non-default Authorizer."); } @Override diff --git a/server/src/main/java/io/druid/server/security/NoopAuthorizationManager.java b/server/src/main/java/io/druid/server/security/NoopAuthorizer.java similarity index 93% rename from server/src/main/java/io/druid/server/security/NoopAuthorizationManager.java rename to server/src/main/java/io/druid/server/security/NoopAuthorizer.java index e2a93527ba7a..4b8ddc53854b 100644 --- a/server/src/main/java/io/druid/server/security/NoopAuthorizationManager.java +++ b/server/src/main/java/io/druid/server/security/NoopAuthorizer.java @@ -19,7 +19,7 @@ package io.druid.server.security; -public class NoopAuthorizationManager implements AuthorizationManager +public class NoopAuthorizer implements Authorizer { @Override public Access authorize(String identity, Resource resource, Action action) diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index 6a5c00704fa0..d83310044082 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -52,10 +52,10 @@ import io.druid.server.log.RequestLogger; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.router.QueryHostFinder; -import io.druid.server.security.AuthorizationManagerMapper; -import io.druid.server.security.NoopAuthorizationManager; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.NoopAuthorizer; import org.eclipse.jetty.client.HttpClient; -import io.druid.server.security.AuthorizationManager; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -114,13 +114,13 @@ public void configure(Binder binder) binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", null, null, new ServerConfig()) ); binder.bind(JettyServerInitializer.class).to(ProxyJettyServerInit.class).in(LazySingleton.class); - binder.bind(AuthorizationManagerMapper.class).toInstance( - new AuthorizationManagerMapper(null) { + binder.bind(AuthorizerMapper.class).toInstance( + new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new NoopAuthorizationManager(); + return new NoopAuthorizer(); } } ); diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index fda855ce75f5..bfa3984383ff 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -45,8 +45,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -142,7 +142,7 @@ public void setup() new NoopRequestLogger(), serverConfig, new AuthConfig(), - new AuthorizationManagerMapper(null) + new AuthorizerMapper(null) ), jsonMapper, jsonMapper, @@ -211,11 +211,11 @@ public void testSecuredQuery() throws Exception EasyMock.replay(testServletRequest); - AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new AuthorizationManager() + return new Authorizer() { @Override public Access authorize(String identity, Resource resource, Action action) @@ -289,11 +289,11 @@ public void testSecuredGetServer() throws Exception EasyMock.replay(testServletRequest); - AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new AuthorizationManager() + return new Authorizer() { @Override public Access authorize(String identity, Resource resource, Action action) @@ -417,12 +417,12 @@ public void testDenySecuredGetServer() throws Exception EasyMock.replay(testServletRequest); - AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) + AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new AuthorizationManager() + return new Authorizer() { @Override public Access authorize(String identity, Resource resource, Action action) diff --git a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java index 13bd0f3e7830..f1409e91f090 100644 --- a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java +++ b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java @@ -29,8 +29,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; @@ -156,11 +156,11 @@ public void testSecuredGetFullQueryableDataSources() throws Exception ).atLeastOnce(); EasyMock.replay(inventoryView, server, request); - AuthorizationManagerMapper authMapper = new AuthorizationManagerMapper(null) { + AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new AuthorizationManager() + return new Authorizer() { @Override public Access authorize(String identity, Resource resource, Action action) diff --git a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java index 3d70f18cda24..0e0c378d4850 100644 --- a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java +++ b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java @@ -36,8 +36,8 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; import org.easymock.EasyMock; @@ -57,19 +57,19 @@ public class ResourceFilterTestHelper { public HttpServletRequest req; - public AuthorizationManagerMapper authorizationManagerMapper; + public AuthorizerMapper authorizerMapper; public ContainerRequest request; public void setUp(ResourceFilter resourceFilter) throws Exception { req = EasyMock.createStrictMock(HttpServletRequest.class); request = EasyMock.createStrictMock(ContainerRequest.class); - authorizationManagerMapper = EasyMock.createStrictMock(AuthorizationManagerMapper.class); + authorizerMapper = EasyMock.createStrictMock(AuthorizerMapper.class); // Memory barrier synchronized (this) { ((AbstractResourceFilter) resourceFilter).setReq(req); - ((AbstractResourceFilter) resourceFilter).setAuthorizationManagerMapper(authorizationManagerMapper); + ((AbstractResourceFilter) resourceFilter).setAuthorizerMapper(authorizerMapper); } } @@ -114,10 +114,10 @@ public MultivaluedMap getMatrixParameters() EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("namespace").atLeastOnce(); req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, authCheckResult); EasyMock.expectLastCall().anyTimes(); - EasyMock.expect(authorizationManagerMapper.getAuthorizationManager( + EasyMock.expect(authorizerMapper.getAuthorizer( EasyMock.anyString() )).andReturn( - new AuthorizationManager() + new Authorizer() { @Override public Access authorize(String identity, Resource resource, Action action) @@ -139,9 +139,9 @@ public static Collection getRequestPaths(final Class clazz) return getRequestPaths(clazz, ImmutableList.>of(), ImmutableList.>of()); } - public static Collection getRequestPathsWithAuthorizationManager(final Class clazz) + public static Collection getRequestPathsWithAuthorizer(final Class clazz) { - return getRequestPaths(clazz, ImmutableList.>of(AuthorizationManagerMapper.class), ImmutableList.>of()); + return getRequestPaths(clazz, ImmutableList.>of(AuthorizerMapper.class), ImmutableList.>of()); } public static Collection getRequestPaths( diff --git a/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java index 8c62576a4ddb..7ccd7d9e91f2 100644 --- a/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java @@ -56,20 +56,20 @@ public static Collection data() { return ImmutableList.copyOf( Iterables.concat( - getRequestPathsWithAuthorizationManager(CoordinatorResource.class), - getRequestPathsWithAuthorizationManager(DatasourcesResource.class), - getRequestPathsWithAuthorizationManager(BrokerResource.class), - getRequestPathsWithAuthorizationManager(HistoricalResource.class), - getRequestPathsWithAuthorizationManager(IntervalsResource.class), - getRequestPathsWithAuthorizationManager(MetadataResource.class), - getRequestPathsWithAuthorizationManager(RulesResource.class), - getRequestPathsWithAuthorizationManager(ServersResource.class), - getRequestPathsWithAuthorizationManager(TiersResource.class), - getRequestPathsWithAuthorizationManager(ClientInfoResource.class), - getRequestPathsWithAuthorizationManager(CoordinatorDynamicConfigsResource.class), - getRequestPathsWithAuthorizationManager(QueryResource.class), - getRequestPathsWithAuthorizationManager(StatusResource.class), - getRequestPathsWithAuthorizationManager(BrokerQueryResource.class) + getRequestPathsWithAuthorizer(CoordinatorResource.class), + getRequestPathsWithAuthorizer(DatasourcesResource.class), + getRequestPathsWithAuthorizer(BrokerResource.class), + getRequestPathsWithAuthorizer(HistoricalResource.class), + getRequestPathsWithAuthorizer(IntervalsResource.class), + getRequestPathsWithAuthorizer(MetadataResource.class), + getRequestPathsWithAuthorizer(RulesResource.class), + getRequestPathsWithAuthorizer(ServersResource.class), + getRequestPathsWithAuthorizer(TiersResource.class), + getRequestPathsWithAuthorizer(ClientInfoResource.class), + getRequestPathsWithAuthorizer(CoordinatorDynamicConfigsResource.class), + getRequestPathsWithAuthorizer(QueryResource.class), + getRequestPathsWithAuthorizer(StatusResource.class), + getRequestPathsWithAuthorizer(BrokerQueryResource.class) ) ); } @@ -102,17 +102,17 @@ public void setUp() throws Exception public void testResourcesFilteringAccess() { setUpMockExpectations(requestPath, true, requestMethod); - EasyMock.replay(req, request, authorizationManagerMapper); + EasyMock.replay(req, request, authorizerMapper); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); resourceFilter.getRequestFilter().filter(request); - EasyMock.verify(req, request, authorizationManagerMapper); + EasyMock.verify(req, request, authorizerMapper); } @Test(expected = WebApplicationException.class) public void testResourcesFilteringNoAccess() { setUpMockExpectations(requestPath, false, requestMethod); - EasyMock.replay(req, request, authorizationManagerMapper); + EasyMock.replay(req, request, authorizerMapper); Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); try { resourceFilter.getRequestFilter().filter(request); @@ -121,16 +121,16 @@ public void testResourcesFilteringNoAccess() Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), e.getResponse().getStatus()); throw e; } - EasyMock.verify(req, request, authorizationManagerMapper); + EasyMock.verify(req, request, authorizerMapper); } @Test public void testResourcesFilteringBadPath() { - EasyMock.replay(req, request, authorizationManagerMapper); + EasyMock.replay(req, request, authorizerMapper); final String badRequestPath = requestPath.replaceAll("\\w+", "droid"); Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath)); - EasyMock.verify(req, request, authorizationManagerMapper); + EasyMock.verify(req, request, authorizerMapper); } } diff --git a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java index 91d893115cd6..b03092159ea7 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java @@ -43,9 +43,9 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.jetty.JettyBindings; import io.druid.server.initialization.jetty.JettyServerInitializer; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; -import io.druid.server.security.NoopAuthorizationManager; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.NoopAuthorizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -78,13 +78,13 @@ public void configure(Binder binder) Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); - binder.bind(AuthorizationManagerMapper.class).toInstance( - new AuthorizationManagerMapper(null) { + binder.bind(AuthorizerMapper.class).toInstance( + new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new NoopAuthorizationManager(); + return new NoopAuthorizer(); } } ); diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 74234527b498..d255a7642df5 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -41,9 +41,9 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.initialization.jetty.ServletFilterHolder; -import io.druid.server.security.AuthorizationManager; -import io.druid.server.security.AuthorizationManagerMapper; -import io.druid.server.security.NoopAuthorizationManager; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.NoopAuthorizer; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Server; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -129,13 +129,13 @@ public EnumSet getDispatcherType() Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); - binder.bind(AuthorizationManagerMapper.class).toInstance( - new AuthorizationManagerMapper(null) { + binder.bind(AuthorizerMapper.class).toInstance( + new AuthorizerMapper(null) { @Override - public AuthorizationManager getAuthorizationManager(String namespace) + public Authorizer getAuthorizer(String namespace) { - return new NoopAuthorizationManager(); + return new NoopAuthorizer(); } } ); diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java index 6f628cde8ab0..a9d6421a7ad9 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java @@ -28,7 +28,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.sql.calcite.rel.DruidConvention; import io.druid.sql.calcite.rel.DruidRel; @@ -68,19 +68,19 @@ public class DruidPlanner implements Closeable private final Planner planner; private final PlannerContext plannerContext; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; public DruidPlanner( final Planner planner, final PlannerContext plannerContext, final AuthConfig authConfig, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { this.planner = planner; this.plannerContext = plannerContext; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } public PlannerResult plan(final String sql) throws SqlParseException, ValidationException, RelConversionException @@ -154,7 +154,7 @@ private PlannerResult planWithDruidConvention( request, datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); } else { authResult = AuthorizationUtils.authorizeAllResourceActions( @@ -162,7 +162,7 @@ private PlannerResult planWithDruidConvention( AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, user, namespace, - authorizationManagerMapper + authorizerMapper ); } @@ -232,7 +232,7 @@ public void visit(RelNode node, int ordinal, RelNode parent) req, datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizationManagerMapper + authorizerMapper ); } else { return AuthorizationUtils.authorizeAllResourceActions( @@ -240,7 +240,7 @@ public void visit(RelNode node, int ordinal, RelNode parent) AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, user, namespace, - authorizationManagerMapper + authorizerMapper ); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java index 57b1e852815e..4ce30b35ab19 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java @@ -23,7 +23,7 @@ import io.druid.math.expr.ExprMacroTable; import io.druid.server.QueryLifecycleFactory; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.schema.DruidSchema; import org.apache.calcite.avatica.util.Casing; @@ -57,7 +57,7 @@ public class PlannerFactory private final PlannerConfig plannerConfig; private final AuthConfig authConfig; - private final AuthorizationManagerMapper authorizationManagerMapper; + private final AuthorizerMapper authorizerMapper; @Inject public PlannerFactory( @@ -67,7 +67,7 @@ public PlannerFactory( final ExprMacroTable macroTable, final PlannerConfig plannerConfig, final AuthConfig authConfig, - final AuthorizationManagerMapper authorizationManagerMapper + final AuthorizerMapper authorizerMapper ) { this.druidSchema = druidSchema; @@ -76,7 +76,7 @@ public PlannerFactory( this.macroTable = macroTable; this.plannerConfig = plannerConfig; this.authConfig = authConfig; - this.authorizationManagerMapper = authorizationManagerMapper; + this.authorizerMapper = authorizerMapper; } public DruidPlanner createPlanner(final Map queryContext) @@ -98,6 +98,6 @@ public DruidPlanner createPlanner(final Map queryContext) .typeSystem(DruidTypeSystem.INSTANCE) .build(); - return new DruidPlanner(Frameworks.getPlanner(frameworkConfig), plannerContext, authConfig, authorizationManagerMapper); + return new DruidPlanner(Frameworks.getPlanner(frameworkConfig), plannerContext, authConfig, authorizerMapper); } } diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index d4e4a3e65bda..f434f1401917 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -36,7 +36,7 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -129,7 +129,7 @@ public void setUp() throws Exception macroTable, plannerConfig, new AuthConfig(), - new AuthorizationManagerMapper(null) + new AuthorizerMapper(null) ), AVATICA_CONFIG, new AuthConfig(), @@ -572,7 +572,7 @@ public int getMaxRowsPerFrame() macroTable, plannerConfig, new AuthConfig(), - new AuthorizationManagerMapper(null) + new AuthorizerMapper(null) ), smallFrameConfig, new AuthConfig(), diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index 109de6959a00..f7017868b714 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -88,7 +88,7 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.log.NoopRequestLogger; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthorizationManagerMapper; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.SqlOperatorConversion; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -313,7 +313,7 @@ public > QueryToolChest getToolChest new NoopRequestLogger(), new ServerConfig(), new AuthConfig(), - new AuthorizationManagerMapper(null) + new AuthorizerMapper(null) ); } From 4c365fe3cc7039fbb1d66083676c3f0e5d6aed4e Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 7 Aug 2017 16:56:05 -0700 Subject: [PATCH 15/34] Fix authorization denials for empty supervisor list --- .../indexing/overlord/supervisor/SupervisorResource.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java index 35bc4bcb2d97..c7c69c242d4d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.inject.Inject; @@ -136,6 +137,13 @@ public Response apply(final SupervisorManager manager) } } } + + AuthorizationUtils.authorizeAllResourceActions( + req, + Lists.newArrayList(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizerMapper + ); } else { supervisorIds = manager.getSupervisorIds(); } From 09455cfafe19d17597f630489f4b511327850230 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 16 Aug 2017 15:13:46 -0700 Subject: [PATCH 16/34] Address some PR comments --- .../kerberos/KerberosAuthenticator.java | 6 ----- .../druid/server/security/Authenticator.java | 5 ---- .../AuthenticatorHttpClientWrapper.java | 22 +++++----------- .../io/druid/server/security/Authorizer.java | 8 ++++++ .../server/security/NoopAuthenticator.java | 6 ----- .../security/SecuritySanityCheckFilter.java | 26 +++++++++++-------- 6 files changed, 29 insertions(+), 44 deletions(-) diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index a5efa6be0af1..ebf0f91869e3 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -105,12 +105,6 @@ public KerberosAuthenticator( this.node = node; } - @Override - public String getTypeName() - { - return "kerberos"; - } - @Override public String getNamespace() { diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index 344f85df1221..ab23308120cf 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -42,11 +42,6 @@ */ public interface Authenticator extends ServletFilterHolder { - /** - * @return The type name of this authenticator. Should be identical to the JsonTypeInfo type. - */ - public String getTypeName(); - /** * @return The namespace associated with this Authenticator. This will be used for choosing the correct * Authorizer for authorizing requests that have been authenticated by this Authenticator. diff --git a/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java b/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java index 5c0d8e09599f..c054c2607547 100644 --- a/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java +++ b/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java @@ -21,10 +21,9 @@ import com.google.common.base.Preconditions; import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.name.Names; import com.metamx.http.client.HttpClient; -import io.druid.java.util.common.ISE; - -import java.util.List; /** * Singleton utility object that creates escalated HttpClients using a configuration-specified Authenticator's @@ -52,20 +51,11 @@ public AuthenticatorHttpClientWrapper( "Auth is enabled but no authenticators have been configured." ); - List authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); String internalAuthenticatorName = authConfig.getInternalAuthenticator(); - for (Authenticator authenticator : authenticators) { - if (authenticator.getTypeName().equals(internalAuthenticatorName)) { - internalAuthenticator = authenticator; - break; - } - } - if (internalAuthenticator == null) { - throw new ISE("Could not locate internal authenticator with type name: %s", internalAuthenticatorName); - } + internalAuthenticator = injector.getInstance(Key.get( + Authenticator.class, + Names.named(internalAuthenticatorName) + )); } } diff --git a/server/src/main/java/io/druid/server/security/Authorizer.java b/server/src/main/java/io/druid/server/security/Authorizer.java index 359bee70cb4e..2eeaa5e07131 100644 --- a/server/src/main/java/io/druid/server/security/Authorizer.java +++ b/server/src/main/java/io/druid/server/security/Authorizer.java @@ -58,10 +58,18 @@ public interface Authorizer */ String getNamespace(); + /** + * Authorizers are registered with an AuthorizerMapper. The AuthorizerMapper is lifecycle managed and will + * call start() on each of its registered Authorizers in the AuthorizerMapper's start() method. + */ default void start() { } + /** + * Authorizers are registered with an AuthorizerMapper. The AuthorizerMapper is lifecycle managed and will + * call stop() on each of its registered Authorizers in the AuthorizerMapper's stop() method. + */ default void stop() { } diff --git a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java index 6a1ed4e39204..49fe647bad21 100644 --- a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java +++ b/server/src/main/java/io/druid/server/security/NoopAuthenticator.java @@ -58,12 +58,6 @@ public EnumSet getDispatcherType() return null; } - @Override - public String getTypeName() - { - return "noop"; - } - @Override public String getNamespace() { diff --git a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java index 0ae79969f3f3..50e128783c75 100644 --- a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java @@ -40,19 +40,30 @@ public class SecuritySanityCheckFilter implements Filter { private static final Logger log = new Logger(SecuritySanityCheckFilter.class); - private final ObjectMapper jsonMapper; + private final String unauthorizedMessage; public SecuritySanityCheckFilter( ObjectMapper jsonMapper ) { - this.jsonMapper = jsonMapper; + try { + QueryInterruptedException unauthorizedError = new QueryInterruptedException( + QueryInterruptedException.UNAUTHORIZED, + null, + null, + DruidNode.getDefaultHost() + ); + unauthorizedError.setStackTrace(new StackTraceElement[0]); + this.unauthorizedMessage = jsonMapper.writeValueAsString(unauthorizedError); + } + catch (Exception e) { + throw new RuntimeException(e); + } } @Override public void init(FilterConfig filterConfig) throws ServletException { - } @Override @@ -61,20 +72,13 @@ public void doFilter( ) throws IOException, ServletException { HttpServletResponse httpResponse = (HttpServletResponse) response; - QueryInterruptedException unauthorizedError = new QueryInterruptedException( - QueryInterruptedException.UNAUTHORIZED, - null, - null, - DruidNode.getDefaultHost() - ); - unauthorizedError.setStackTrace(new StackTraceElement[0]); OutputStream out = httpResponse.getOutputStream(); // make sure the original request isn't trying to fake the auth token checks Boolean authInfoChecked = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); String authToken = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); if (authInfoChecked != null || authToken != null) { - sendJsonError(httpResponse, Response.SC_FORBIDDEN, jsonMapper.writeValueAsString(unauthorizedError), out); + sendJsonError(httpResponse, Response.SC_FORBIDDEN, unauthorizedMessage, out); out.close(); return; } From 655ffacc8a97244af16ef4918424daf41fc40178 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 11:01:13 -0700 Subject: [PATCH 17/34] Address more PR comments --- .../druid/benchmark/query/SqlBenchmark.java | 19 ++- docs/content/configuration/auth.md | 59 +++++-- .../AuthenticationKerberosConfig.java | 97 ------------ .../kerberos/DruidKerberosModule.java | 13 +- .../security/kerberos/DruidKerberosUtil.java | 19 ++- .../kerberos/KerberosAuthenticator.java | 82 ++++++---- .../security/kerberos/KerberosHttpClient.java | 10 +- .../kerberos/KerberosHttpClientProvider.java | 55 ------- .../KerberosJettyHttpClientProvider.java | 22 +-- .../AuthenticationKerberosConfigTest.java | 20 +-- .../sql/QuantileSqlAggregatorTest.java | 30 +++- .../druid/indexing/kafka/KafkaIndexTask.java | 94 +++++------ .../kafka/supervisor/KafkaSupervisorTest.java | 3 +- .../overlord/http/OverlordResource.java | 147 ++++++++---------- .../security/SupervisorResourceFilter.java | 86 +++++----- .../http/security/TaskResourceFilter.java | 78 +++++----- .../supervisor/SupervisorResource.java | 110 ++++++------- .../overlord/http/OverlordResourceTest.java | 23 ++- .../indexing/overlord/http/OverlordTest.java | 25 ++- .../supervisor/SupervisorResourceTest.java | 75 ++++++++- .../guice/security/AuthenticatorModule.java | 8 +- .../guice/security/AuthorizerModule.java | 8 +- .../druid/initialization/Initialization.java | 10 +- .../io/druid/server/ClientInfoResource.java | 21 +-- .../java/io/druid/server/QueryLifecycle.java | 78 ++++------ .../druid/server/QueryLifecycleFactory.java | 3 - .../java/io/druid/server/QueryResource.java | 33 ++-- .../server/http/DatasourcesResource.java | 14 +- .../druid/server/http/IntervalsResource.java | 27 ++-- .../druid/server/http/InventoryViewUtils.java | 8 +- .../druid/server/http/MetadataResource.java | 22 +-- .../http/security/ConfigResourceFilter.java | 33 ++-- .../security/DatasourceResourceFilter.java | 32 ++-- .../http/security/RulesResourceFilter.java | 58 ++++--- .../http/security/StateResourceFilter.java | 32 ++-- ...ule.java => AllowAllAuthorizerModule.java} | 8 +- .../AuthenticatorHttpClientWrapperModule.java | 13 +- .../AuthenticatorMapperModule.java | 120 ++++++++++++++ .../AuthorizerMapperModule.java | 41 +++-- .../cache/LookupCoordinatorManager.java | 6 +- .../java/io/druid/server/security/Access.java | 2 + .../java/io/druid/server/security/Action.java | 3 +- ...icator.java => AllowAllAuthenticator.java} | 27 ++-- ...uthorizer.java => AllowAllAuthorizer.java} | 11 +- .../io/druid/server/security/AuthConfig.java | 34 ++-- .../server/security/AuthenticationResult.java | 61 ++++++++ .../server/security/AuthenticationUtils.java | 21 --- .../druid/server/security/Authenticator.java | 22 +-- .../AuthenticatorHttpClientWrapper.java | 35 +---- .../server/security/AuthenticatorMapper.java | 80 ++++++++++ .../server/security/AuthorizationUtils.java | 97 +++++------- .../io/druid/server/security/Authorizer.java | 19 +-- .../server/security/AuthorizerMapper.java | 4 +- ...Authorizer.java => DenyAllAuthorizer.java} | 12 +- .../PreResponseAuthorizationCheckFilter.java | 86 +++++----- .../druid/server/security/ResourceType.java | 3 +- .../security/SecuritySanityCheckFilter.java | 6 +- .../security/UnsecuredResourceFilter.java | 2 +- .../io/druid/client/BrokerServerViewTest.java | 4 +- .../AsyncQueryForwardingServletTest.java | 6 +- .../io/druid/server/QueryResourceTest.java | 64 ++++---- .../DruidCoordinatorSegmentMergerTest.java | 4 +- .../server/http/DatasourcesResourceTest.java | 63 +++++--- .../server/http/IntervalsResourceTest.java | 56 ++++--- ...eResponseAuthorizationCheckFilterTest.java | 21 +-- .../security/ResourceFilterTestHelper.java | 18 +-- .../SecuritySanityCheckFilterTest.java | 11 +- .../server/initialization/JettyQosTest.java | 6 +- .../server/initialization/JettyTest.java | 6 +- .../cache/LookupCoordinatorManagerTest.java | 17 +- .../router/TieredBrokerHostSelectorTest.java | 13 +- .../main/java/io/druid/cli/CliOverlord.java | 37 ++--- .../CoordinatorJettyServerInitializer.java | 36 ++--- .../MiddleManagerJettyServerInitializer.java | 31 ++-- .../cli/QueryJettyServerInitializer.java | 30 ++-- .../cli/RouterJettyServerInitializer.java | 31 ++-- .../io/druid/sql/avatica/DruidConnection.java | 6 +- .../java/io/druid/sql/avatica/DruidMeta.java | 61 +++----- .../io/druid/sql/avatica/DruidStatement.java | 8 +- .../sql/calcite/planner/DruidPlanner.java | 94 ++++++----- .../sql/calcite/planner/PlannerContext.java | 10 +- .../sql/calcite/planner/PlannerFactory.java | 20 ++- .../io/druid/sql/calcite/rel/QueryMaker.java | 7 +- .../druid/sql/calcite/schema/DruidSchema.java | 19 ++- .../java/io/druid/sql/http/SqlResource.java | 2 +- .../sql/avatica/DruidAvaticaHandlerTest.java | 48 +++++- .../druid/sql/avatica/DruidStatementTest.java | 28 +++- .../druid/sql/calcite/CalciteQueryTest.java | 54 +++++-- .../calcite/expression/ExpressionsTest.java | 10 ++ .../sql/calcite/http/SqlResourceTest.java | 41 ++++- .../sql/calcite/schema/DruidSchemaTest.java | 10 +- .../druid/sql/calcite/util/CalciteTests.java | 17 +- 92 files changed, 1661 insertions(+), 1395 deletions(-) delete mode 100644 extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java delete mode 100644 extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClientProvider.java rename server/src/main/java/io/druid/server/initialization/{NoopAuthorizerModule.java => AllowAllAuthorizerModule.java} (86%) create mode 100644 server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java rename server/src/main/java/io/druid/server/security/{NoopAuthenticator.java => AllowAllAuthenticator.java} (80%) rename server/src/main/java/io/druid/server/security/{NoopAuthorizer.java => AllowAllAuthorizer.java} (79%) create mode 100644 server/src/main/java/io/druid/server/security/AuthenticationResult.java create mode 100644 server/src/main/java/io/druid/server/security/AuthenticatorMapper.java rename server/src/main/java/io/druid/server/security/{DefaultAuthorizer.java => DenyAllAuthorizer.java} (75%) diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java index baa5b40fc1ec..f030753553fe 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java @@ -39,7 +39,13 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.segment.QueryableIndex; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.planner.PlannerFactory; @@ -67,6 +73,7 @@ import java.io.File; import java.util.ArrayList; import java.util.Arrays; +import java.util.Map; import java.util.concurrent.TimeUnit; /** @@ -113,7 +120,8 @@ public void setup() throws Exception final PlannerConfig plannerConfig = new PlannerConfig(); this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index); - + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); plannerFactory = new PlannerFactory( CalciteTests.createMockSchema(walker, plannerConfig), CalciteTests.createMockQueryLifecycleFactory(walker), @@ -121,7 +129,14 @@ public void setup() throws Exception CalciteTests.createExprMacroTable(), plannerConfig, new AuthConfig(), - null + new AuthenticatorMapper(defaultMap, "allowAll"), + new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + } ); groupByQuery = GroupByQuery .builder() diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index 1af50350fbdb..31c7e2278b5a 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -8,24 +8,26 @@ layout: doc_page |--------|-----------|--------|--------|--------| |`druid.auth.enabled`|boolean|Determines if authentication and authorization checks will be performed on requests.|false|no| |`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|null|yes, if auth enabled| -|`druid.auth.internalAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes, if auth enabled| +|`druid.auth.escalatedAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes, if auth enabled| |`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |null|yes, if auth enabled| ## Enabling Authentication/Authorization ## Authentication Chain -Authentication decisions are handled by a chain of Authenticator instances. A request will be checked by Authenticators in the sequence defined by the `druid.auth.authenticationChain` file. +Authentication decisions are handled by a chain of Authenticator instances. A request will be checked by Authenticators in the sequence defined by the `druid.auth.authenticationChain`. Authenticator implementions are provided by extensions. For example, the following authentication chain definition enables the Kerberos and HTTP Basic authenticators, from the `druid-kerberos` and `druid-basic-security` core extensions, respectively: -```json -["kerberos", "basic"] ``` +druid.auth.authenticationChain=["kerberos", "basic"] +``` + +A request will pass through all Authenticators in the chain, unless one of the Authenticators sends an HTTP error response. If no Authenticator in the chain successfully authenticated a request, an HTTP error response will be sent. ## Internal Authenticator -The `druid.auth.internalAuthenticator` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a broker node communicates with historical nodes for query processing). +The `druid.auth.escalatedAuthenticator` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a broker node communicates with historical nodes for query processing). The Authenticator chosen for this property must also be present in `druid.auth.authenticationChain`. @@ -36,20 +38,55 @@ There are two built-in Authorizers, "default" and "noop". Other implementations For example, the following authorizers definition enables the "basic" implementation from `druid-basic-security`: -```json -["basic"] +``` +druid.auth.authorizers=["basic"] ``` + +Only a single Authorizer will authorize any given request. + ### Default Authorizer The default Authorizer with type name "default" rejects all requests. ### No-op Authorizer The no-op Authorizer with type name "noop" accepts all requests. +## Authenticator to Authorizer Routing + +When an Authenticator successfully authenticates a request, it must attach a AuthenticationResult to the request, containing an information about the identity of the requester, as well as the name of the Authorizer that should authorize the authenticated request. + +An Authenticator implementation should provide some means through configuration to allow users to select what Authorizer(s) the Authenticator should route requests to. + +## Internal System User + +Internal requests between Druid nodes (non-user initiated communications) need to have authentication credentials attached. These requests should be run as an "internal system user". + +We recommend that extension implementers follow the guidelines below regarding the "internal system user", for maximum compatibility between different Authenticator and Authorizer implementations. + +### Authorizer Internal System User Handling + +Authorizers implementations must recognize and authorize an identity for the "internal system user", with unrestricted permissions. + +We recommend that this "internal system user" be represented by the identity string "__DRUID_INTERNAL_SYSTEM". This is a guideline only and not enforced; if an Authorizer needs to use a different identity string format, it is free to do so. + +Allowing the user to redefine what identity string represents the internal system user is also recommended. + +### Authenticator Internal System User Handling + +Authenticators must implement two methods related to the internal system user: + +```java + public HttpClient createEscalatedClient(HttpClient baseClient); + + public AuthenticationResult createEscalatedAuthenticationResult(); +``` + +`createEscalatedClient` returns an wrapped HttpClient that attaches the credentials of the "internal system user" to requests. + +`createEscalatedAuthenticationResult` returns an AuthenticationResult containing the identity of the "internal system user". + +As with Authenticators, we recommend that the "internal system user" be represented by default with the identity string "__DRUID_INTERNAL_SYSTEM". This is a guideline and not enforced. -## Namespaces -Authenticator and Authorizer implementations are linked through a namespace string. Authenticators tag an authenticated request with a namespace, which is used to route the authenticated request to the Authorizer implementation that registered itself with a matching namespace. +We also recommend that Authenticator implementations allow the user to redefine the identity string used for the internal system users, if feasible. -This is to support cases where an Authorizer implementation is only intended to authorize requests from a specific authenticator (an implementation may have assumptions about the user name format, for example). -The details of namespace configuration are left for implementors of Authenticator and Authorizer to decide. \ No newline at end of file diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java deleted file mode 100644 index be695c92ccd0..000000000000 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/AuthenticationKerberosConfig.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.security.kerberos; - - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -public class AuthenticationKerberosConfig -{ - @JsonProperty - private final String principal; - - @JsonProperty - private final String keytab; - - @JsonProperty - private final String namespace; - - @JsonCreator - public AuthenticationKerberosConfig( - @JsonProperty("principal") String principal, - @JsonProperty("keytab") String keytab, - @JsonProperty("namespace") String namespace - ) - { - this.principal = principal; - this.keytab = keytab; - this.namespace = namespace; - } - - @JsonProperty - public String getPrincipal() - { - return principal; - } - - @JsonProperty - public String getKeytab() - { - return keytab; - } - - @JsonProperty - public String getNamespace() - { - return namespace; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - AuthenticationKerberosConfig that = (AuthenticationKerberosConfig) o; - - if (getPrincipal() != null ? !getPrincipal().equals(that.getPrincipal()) : that.getPrincipal() != null) { - return false; - } - if (getKeytab() != null ? !getKeytab().equals(that.getKeytab()) : that.getKeytab() != null) { - return false; - } - return getNamespace() != null ? getNamespace().equals(that.getNamespace()) : that.getNamespace() == null; - - } - - @Override - public int hashCode() - { - int result = getPrincipal() != null ? getPrincipal().hashCode() : 0; - result = 31 * result + (getKeytab() != null ? getKeytab().hashCode() : 0); - result = 31 * result + (getNamespace() != null ? getNamespace().hashCode() : 0); - return result; - } -} diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java index be6d36271f58..e6c85fcc4f3e 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java @@ -23,14 +23,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import com.google.inject.Provides; -import com.google.inject.name.Named; -import io.druid.guice.JsonConfigProvider; -import io.druid.guice.ManageLifecycle; -import io.druid.guice.annotations.Self; import io.druid.initialization.DruidModule; -import io.druid.server.DruidNode; -import io.druid.server.security.Authenticator; import java.util.List; @@ -52,10 +45,11 @@ public List getJacksonModules() @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", AuthenticationKerberosConfig.class); - JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); + //JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", AuthenticationKerberosConfig.class); + //JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); } + /* @Provides @ManageLifecycle @Named("kerberos") @@ -67,4 +61,5 @@ public Authenticator getAuthenticator( { return new KerberosAuthenticator(authenticationKerberosConfig, spnegoFilterConfig, druidNode); } + */ } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java index 872f820c38c5..1487e76d5857 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosUtil.java @@ -91,24 +91,27 @@ public static String kerberosChallenge(String server) throws AuthenticationExcep } } - public static void authenticateIfRequired(AuthenticationKerberosConfig config) + public static void authenticateIfRequired(String internalClientPrincipal, String internalClientKeytab) throws IOException { - String principal = config.getPrincipal(); - String keytab = config.getKeytab(); - if (!Strings.isNullOrEmpty(principal) && !Strings.isNullOrEmpty(keytab)) { + if (!Strings.isNullOrEmpty(internalClientPrincipal) && !Strings.isNullOrEmpty(internalClientKeytab)) { Configuration conf = new Configuration(); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); UserGroupInformation.setConfiguration(conf); try { if (UserGroupInformation.getCurrentUser().hasKerberosCredentials() == false - || !UserGroupInformation.getCurrentUser().getUserName().equals(principal)) { - log.info("trying to authenticate user [%s] with keytab [%s]", principal, keytab); - UserGroupInformation.loginUserFromKeytab(principal, keytab); + || !UserGroupInformation.getCurrentUser().getUserName().equals(internalClientPrincipal)) { + log.info("trying to authenticate user [%s] with keytab [%s]", internalClientPrincipal, internalClientKeytab); + UserGroupInformation.loginUserFromKeytab(internalClientPrincipal, internalClientKeytab); } } catch (IOException e) { - throw new ISE(e, "Failed to authenticate user principal [%s] with keytab [%s]", principal, keytab); + throw new ISE( + e, + "Failed to authenticate user principal [%s] with keytab [%s]", + internalClientPrincipal, + internalClientKeytab + ); } } } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index ebf0f91869e3..71b1ae823e17 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -19,8 +19,8 @@ package io.druid.security.kerberos; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Throwables; import com.metamx.http.client.HttpClient; @@ -29,6 +29,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.server.DruidNode; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authenticator; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.security.SecurityUtil; @@ -75,6 +76,7 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Properties; import java.util.Random; @@ -88,27 +90,39 @@ public class KerberosAuthenticator implements Authenticator private static final Logger log = new Logger(KerberosAuthenticator.class); private static final Pattern HADOOP_AUTH_COOKIE_REGEX = Pattern.compile(".*p=(\\S+)&t=.*"); - private final SpnegoFilterConfig spnegoConfig; - private final AuthenticationKerberosConfig authConfig; private final DruidNode node; + private final String serverPrincipal; + private final String serverKeytab; + private final String internalClientPrincipal; + private final String internalClientKeytab; + private final String authToLocal; + private final List excludedPaths; + private final String cookieSignatureSecret; + private final String authorizerName; private LoginContext loginContext; @JsonCreator public KerberosAuthenticator( - @JacksonInject AuthenticationKerberosConfig authConfig, - @JacksonInject SpnegoFilterConfig spnegoConfig, + @JsonProperty("serverPrincipal") String serverPrincipal, + @JsonProperty("serverKeytab") String serverKeytab, + @JsonProperty("internalClientPrincipal") String internalClientPrincipal, + @JsonProperty("internalClientKeytab") String internalClientKeytab, + @JsonProperty("authToLocal") String authToLocal, + @JsonProperty("excludedPaths") List excludedPaths, + @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret, + @JsonProperty("authorizerName") String authorizerName, @Self DruidNode node ) { - this.spnegoConfig = spnegoConfig; - this.authConfig = authConfig; this.node = node; - } - - @Override - public String getNamespace() - { - return authConfig.getNamespace(); + this.serverPrincipal = serverPrincipal; + this.serverKeytab = serverKeytab; + this.internalClientPrincipal = internalClientPrincipal; + this.internalClientKeytab = internalClientKeytab; + this.authToLocal = authToLocal; + this.excludedPaths = excludedPaths; + this.cookieSignatureSecret = cookieSignatureSecret; + this.authorizerName = authorizerName; } @Override @@ -203,8 +217,8 @@ public void doFilter( { HttpServletRequest httpReq = (HttpServletRequest) request; - // If there's already an auth token, then we have authenticated already, skip this. - if (request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) != null) { + // If there's already an auth result, then we have authenticated already, skip this. + if (request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) != null) { filterChain.doFilter(request, response); return; } @@ -239,8 +253,10 @@ public void doFilter( clientPrincipal = null; } - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN, clientPrincipal); - request.setAttribute(AuthConfig.DRUID_AUTH_NAMESPACE, getNamespace()); + request.setAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT, + new AuthenticationResult(clientPrincipal, authorizerName) + ); } doFilterSuper(request, response, filterChain); @@ -270,7 +286,7 @@ private void doFilterSuper(ServletRequest request, ServletResponse response, Fil if (getAuthenticationHandler().managementOperation(token, httpRequest, httpResponse)) { if (token == null) { if (log.isDebugEnabled()) { - log.debug("Request [{}] triggering authentication", getRequestURL(httpRequest)); + log.debug("Request [{%s}] triggering authentication", getRequestURL(httpRequest)); } token = getAuthenticationHandler().authenticate(httpRequest, httpResponse); if (token != null && token.getExpires() != 0 && @@ -282,7 +298,7 @@ private void doFilterSuper(ServletRequest request, ServletResponse response, Fil if (token != null) { unauthorizedResponse = false; if (log.isDebugEnabled()) { - log.debug("Request [{}] user [{}] authenticated", getRequestURL(httpRequest), token.getUserName()); + log.debug("Request [{%s}] user [{%s}] authenticated", getRequestURL(httpRequest), token.getUserName()); } final AuthenticationToken authToken = token; httpRequest = new HttpServletRequestWrapper(httpRequest) @@ -369,14 +385,14 @@ public Map getInitParameters() try { params.put( "kerberos.principal", - SecurityUtil.getServerPrincipal(spnegoConfig.getPrincipal(), node.getHost()) + SecurityUtil.getServerPrincipal(serverPrincipal, node.getHost()) ); - params.put("kerberos.keytab", spnegoConfig.getKeytab()); + params.put("kerberos.keytab", serverKeytab); //params.put(AuthenticationFilter.AUTH_TYPE, "kerberos"); params.put(AuthenticationFilter.AUTH_TYPE, DruidKerberosAuthenticationHandler.class.getName()); - params.put("kerberos.name.rules", spnegoConfig.getAuthToLocal()); - if (spnegoConfig.getCookieSignatureSecret() != null) { - params.put("signature.secret", spnegoConfig.getCookieSignatureSecret()); + params.put("kerberos.name.rules", authToLocal); + if (cookieSignatureSecret != null) { + params.put("signature.secret", cookieSignatureSecret); } } catch (IOException e) { @@ -404,7 +420,7 @@ public String getAuthChallengeHeader() } @Override - public boolean authenticateJDBCContext(Map context) + public AuthenticationResult authenticateJDBCContext(Map context) { throw new UnsupportedOperationException("JDBC Kerberos auth not supported yet"); } @@ -412,12 +428,18 @@ public boolean authenticateJDBCContext(Map context) @Override public HttpClient createEscalatedClient(HttpClient baseClient) { - return new KerberosHttpClient(baseClient, authConfig); + return new KerberosHttpClient(baseClient, internalClientPrincipal, internalClientKeytab); + } + + @Override + public AuthenticationResult createEscalatedAuthenticationResult() + { + return new AuthenticationResult(internalClientPrincipal, authorizerName); } private boolean isExcluded(String path) { - for (String excluded : spnegoConfig.getExcludedPaths()) { + for (String excluded : excludedPaths) { if (path.startsWith(excluded)) { return true; } @@ -518,7 +540,7 @@ private String getPrincipalFromRequestNew(HttpServletRequest req) for (Object cred : serverCreds) { if (cred instanceof KeyTab) { KeyTab serverKeyTab = (KeyTab) cred; - KerberosPrincipal serverPrincipal = new KerberosPrincipal(spnegoConfig.getPrincipal()); + KerberosPrincipal serverPrincipal = new KerberosPrincipal(this.serverPrincipal); KerberosKey[] serverKeys = serverKeyTab.getKeys(serverPrincipal); for (KerberosKey key : serverKeys) { if (key.getKeyType() == eType) { @@ -562,11 +584,11 @@ private void initializeKerberosLogin() throws ServletException String keytab; try { - principal = SecurityUtil.getServerPrincipal(spnegoConfig.getPrincipal(), node.getHost()); + principal = SecurityUtil.getServerPrincipal(serverPrincipal, node.getHost()); if (principal == null || principal.trim().length() == 0) { throw new ServletException("Principal not defined in configuration"); } - keytab = spnegoConfig.getKeytab(); + keytab = serverKeytab; if (keytab == null || keytab.trim().length() == 0) { throw new ServletException("Keytab not defined in configuration"); } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java index 4930006b181d..979a10fcab5f 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClient.java @@ -47,15 +47,17 @@ public class KerberosHttpClient extends AbstractHttpClient private static final Logger log = new Logger(KerberosHttpClient.class); private final HttpClient delegate; - private final AuthenticationKerberosConfig config; private final CookieManager cookieManager; private final Executor exec = Execs.singleThreaded("test-%s"); + private final String internalClientPrincipal; + private final String internalClientKeytab; - public KerberosHttpClient(HttpClient delegate, AuthenticationKerberosConfig config) + public KerberosHttpClient(HttpClient delegate, String internalClientPrincipal, String internalClientKeytab) { this.delegate = delegate; - this.config = config; this.cookieManager = new CookieManager(); + this.internalClientPrincipal = internalClientPrincipal; + this.internalClientKeytab = internalClientKeytab; } @Override @@ -94,7 +96,7 @@ private void inner_go( uri, cookieManager.getCookieStore().getCookies() ); - DruidKerberosUtil.authenticateIfRequired(config); + DruidKerberosUtil.authenticateIfRequired(internalClientPrincipal, internalClientKeytab); UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); String challenge = currentUser.doAs(new PrivilegedExceptionAction() { diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClientProvider.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClientProvider.java deleted file mode 100644 index 6d4cb6234566..000000000000 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosHttpClientProvider.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.security.kerberos; - -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.google.inject.Provider; -import com.metamx.http.client.HttpClient; -import io.druid.guice.http.AbstractHttpClientProvider; - -public class KerberosHttpClientProvider extends AbstractHttpClientProvider -{ - private final Provider delegateProvider; - private AuthenticationKerberosConfig config; - - public KerberosHttpClientProvider( - Provider delegateProvider - ) - { - this.delegateProvider = delegateProvider; - } - - @Inject - @Override - public void configure(Injector injector) - { - if (delegateProvider instanceof AbstractHttpClientProvider) { - ((AbstractHttpClientProvider) delegateProvider).configure(injector); - } - config = injector.getInstance(AuthenticationKerberosConfig.class); - } - - @Override - public HttpClient get() - { - return new KerberosHttpClient(delegateProvider.get(), config); - } -} diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java index cb8575bf48f6..1a60cfc623fb 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java @@ -19,26 +19,17 @@ package io.druid.security.kerberos; - -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.google.inject.Provider; import io.druid.guice.http.AbstractHttpClientProvider; -import io.druid.java.util.common.logger.Logger; -import org.apache.hadoop.security.UserGroupInformation; import org.eclipse.jetty.client.HttpClient; -import org.eclipse.jetty.client.api.Authentication; -import org.eclipse.jetty.client.api.ContentResponse; -import org.eclipse.jetty.client.api.Request; -import org.eclipse.jetty.util.Attributes; -import org.jboss.netty.handler.codec.http.HttpHeaders; - -import java.net.URI; -import java.security.PrivilegedExceptionAction; public class KerberosJettyHttpClientProvider extends AbstractHttpClientProvider { + @Override + public HttpClient get() + { + return null; + } + /* private static final Logger log = new Logger(KerberosJettyHttpClientProvider.class); private final Provider delegateProvider; @@ -125,4 +116,5 @@ public String run() throws Exception }); return httpClient; } + */ } diff --git a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java index 235513034464..ac20101f11ee 100644 --- a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java +++ b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java @@ -19,29 +19,14 @@ package io.druid.security.kerberos; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.Provides; -import io.druid.guice.ConfigModule; -import io.druid.guice.DruidGuiceExtensions; -import io.druid.guice.JsonConfigProvider; -import io.druid.guice.LazySingleton; -import io.druid.guice.PropertiesModule; -import io.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; import org.junit.Test; -import java.util.Arrays; -import java.util.Properties; - public class AuthenticationKerberosConfigTest { @Test public void testserde() { + /* Injector injector = Guice.createInjector( new Module() { @@ -68,7 +53,6 @@ public ObjectMapper jsonMapper() Assert.assertEquals(props.getProperty("druid.hadoop.security.kerberos.principal"), config.getPrincipal()); Assert.assertEquals(props.getProperty("druid.hadoop.security.kerberos.keytab"), config.getKeytab()); - - + */ } } diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index 86ceea652ce2..79a503946249 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; @@ -44,12 +45,19 @@ import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.virtual.ExpressionVirtualColumn; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerConfig; +import io.druid.sql.calcite.planner.PlannerContext; import io.druid.sql.calcite.planner.PlannerFactory; import io.druid.sql.calcite.planner.PlannerResult; import io.druid.sql.calcite.schema.DruidSchema; @@ -67,6 +75,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Map; public class QuantileSqlAggregatorTest { @@ -128,6 +137,8 @@ public void setUp() throws Exception ImmutableSet.of(new QuantileSqlAggregator()), ImmutableSet.of() ); + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); plannerFactory = new PlannerFactory( druidSchema, CalciteTests.createMockQueryLifecycleFactory(walker), @@ -135,7 +146,14 @@ public void setUp() throws Exception CalciteTests.createExprMacroTable(), plannerConfig, new AuthConfig(), - null + new AuthenticatorMapper(defaultMap, "allowAll"), + new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + } ); } @@ -223,7 +241,10 @@ public void testQuantileOnFloatAndLongs() throws Exception new QuantilePostAggregator("a7", "a5:agg", 0.999f), new QuantilePostAggregator("a8", "a8:agg", 0.50f) )) - .context(ImmutableMap.of("skipEmptyBuckets", true)) + .context(ImmutableMap.of( + "skipEmptyBuckets", true, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + )) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); @@ -283,7 +304,10 @@ public void testQuantileOnComplexColumn() throws Exception new QuantilePostAggregator("a5", "a5:agg", 0.999f), new QuantilePostAggregator("a6", "a4:agg", 0.999f) )) - .context(ImmutableMap.of("skipEmptyBuckets", true)) + .context(ImmutableMap.of( + "skipEmptyBuckets", true, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + )) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 9fc7293ec658..c83acf23a280 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -727,14 +727,13 @@ public Sequence run(final QueryPlus queryPlus, final Map r @Path("/stop") public Response stop(@Context final HttpServletRequest req) { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); } + stopGracefully(); return Response.status(Response.Status.OK).build(); } @@ -744,13 +743,12 @@ public Response stop(@Context final HttpServletRequest req) @Produces(MediaType.APPLICATION_JSON) public Status getStatusHTTP(@Context final HttpServletRequest req) { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.READ); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } + Access access = authorizationCheck(req, Action.READ); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; } + return status; } @@ -764,13 +762,12 @@ public Status getStatus() @Produces(MediaType.APPLICATION_JSON) public Map getCurrentOffsets(@Context final HttpServletRequest req) { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.READ); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } + Access access = authorizationCheck(req, Action.READ); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; } + return getCurrentOffsets(); } @@ -784,13 +781,12 @@ public Map getCurrentOffsets() @Produces(MediaType.APPLICATION_JSON) public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.READ); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } + Access access = authorizationCheck(req, Action.READ); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; } + return getEndOffsets(); } @@ -809,13 +805,11 @@ public Response setEndOffsetsHTTP( @Context final HttpServletRequest req ) throws InterruptedException { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); } return setEndOffsets(offsets, resume); @@ -894,14 +888,13 @@ public Response pauseHTTP( @Context final HttpServletRequest req ) throws InterruptedException { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); } + return pause(timeout); } @@ -956,14 +949,13 @@ public Response pause(final long timeout) throws InterruptedException @Path("/resume") public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .entity("Request authorization failed.") + .build(); } + resume(); return Response.status(Response.Status.OK).build(); } @@ -993,12 +985,10 @@ public void resume() throws InterruptedException @Produces(MediaType.APPLICATION_JSON) public DateTime getStartTime(@Context final HttpServletRequest req) { - if (authConfig.isEnabled()) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } + Access access = authorizationCheck(req, Action.WRITE); + if (!access.isAllowed()) { + log.warn("Authorization failure."); + return null; } return startTime; } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 361e3bbbf0d3..54d137fc2329 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -71,6 +71,7 @@ import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticatorHttpClientWrapper; +import io.druid.server.security.AllowAllAuthenticator; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -1824,7 +1825,7 @@ private KafkaSupervisor getSupervisor( KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( null, null, - new AuthenticatorHttpClientWrapper(new AuthConfig(), null) + new AuthenticatorHttpClientWrapper(new AllowAllAuthenticator()) ) { @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 5f0d21b1ad83..51e59e4f3d37 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -48,12 +48,8 @@ import io.druid.indexing.overlord.autoscaling.ScalingStats; import io.druid.indexing.overlord.http.security.TaskResourceFilter; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; -<<<<<<< HEAD -======= import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.Pair; ->>>>>>> upstream/master import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.EntryExistsException; @@ -139,25 +135,23 @@ public Response taskPost( @Context final HttpServletRequest req ) { - if (authConfig.isEnabled()) { - final String dataSource = task.getDataSource(); - final ResourceAction resourceAction = new ResourceAction( - new Resource(dataSource, ResourceType.DATASOURCE), - Action.WRITE - ); - - Access authResult = AuthorizationUtils.authorizeResourceAction( - req, - resourceAction, - authorizerMapper - ); - - if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .header("Access-Check-Result", authResult) - .entity(ImmutableMap.of("error", "Not authorized.")) - .build(); - } + final String dataSource = task.getDataSource(); + final ResourceAction resourceAction = new ResourceAction( + new Resource(dataSource, ResourceType.DATASOURCE), + Action.WRITE + ); + + Access authResult = AuthorizationUtils.authorizeResourceAction( + req, + resourceAction, + authorizerMapper + ); + + if (!authResult.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN) + .header("Access-Check-Result", authResult) + .entity(ImmutableMap.of("error", "Not authorized.")) + .build(); } return asLeaderWith( @@ -378,29 +372,25 @@ public Collection apply(TaskRunner taskRunner) // off to the runner yet: final List allActiveTasks = taskStorageQueryAdapter.getActiveTasks(); final List activeTasks; - if (authConfig.isEnabled()) { - Function raGenerator = new Function() + Function raGenerator = new Function() + { + @Override + public ResourceAction apply(Task input) { - @Override - public ResourceAction apply(Task input) - { - return new ResourceAction( - new Resource(input.getDataSource(), ResourceType.DATASOURCE), - Action.READ - ); - } - }; - - activeTasks = AuthorizationUtils.filterAuthorizedResources( - req, - allActiveTasks, - raGenerator, - authorizerMapper - ); + return new ResourceAction( + new Resource(input.getDataSource(), ResourceType.DATASOURCE), + Action.READ + ); + } + }; + + activeTasks = AuthorizationUtils.filterAuthorizedResources( + req, + allActiveTasks, + raGenerator, + authorizerMapper + ); - } else { - activeTasks = allActiveTasks; - } final Set runnersKnownTasks = Sets.newHashSet( Iterables.transform( taskRunner.getKnownTasks(), @@ -452,12 +442,7 @@ public Response getPendingTasks(@Context final HttpServletRequest req) @Override public Collection apply(TaskRunner taskRunner) { - if (authConfig.isEnabled()) { - return securedTaskRunnerWorkItem(taskRunner.getPendingTasks(), req); - } else { - return taskRunner.getPendingTasks(); - } - + return securedTaskRunnerWorkItem(taskRunner.getPendingTasks(), req); } } ); @@ -474,11 +459,7 @@ public Response getRunningTasks(@Context final HttpServletRequest req) @Override public Collection apply(TaskRunner taskRunner) { - if (authConfig.isEnabled()) { - return securedTaskRunnerWorkItem(taskRunner.getRunningTasks(), req); - } else { - return taskRunner.getRunningTasks(); - } + return securedTaskRunnerWorkItem(taskRunner.getRunningTasks(), req); } } ); @@ -490,38 +471,34 @@ public Collection apply(TaskRunner taskRunner) public Response getCompleteTasks(@Context final HttpServletRequest req) { final List recentlyFinishedTasks; - if (authConfig.isEnabled()) { - Function raGenerator = new Function() + Function raGenerator = new Function() + { + @Override + public ResourceAction apply(TaskStatus input) { - @Override - public ResourceAction apply(TaskStatus input) - { - final String taskId = input.getId(); - final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); - if (!optionalTask.isPresent()) { - throw new WebApplicationException( - Response.serverError().entity( - String.format("No task information found for task with id: [%s]", taskId) - ).build() - ); - } - - return new ResourceAction( - new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE), - Action.READ + final String taskId = input.getId(); + final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); + if (!optionalTask.isPresent()) { + throw new WebApplicationException( + Response.serverError().entity( + StringUtils.format("No task information found for task with id: [%s]", taskId) + ).build() ); } - }; - - recentlyFinishedTasks = AuthorizationUtils.filterAuthorizedResources( - req, - taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(), - raGenerator, - authorizerMapper - ); - } else { - recentlyFinishedTasks = taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(); - } + + return new ResourceAction( + new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE), + Action.READ + ); + } + }; + + recentlyFinishedTasks = AuthorizationUtils.filterAuthorizedResources( + req, + taskStorageQueryAdapter.getRecentlyFinishedTaskStatuses(), + raGenerator, + authorizerMapper + ); final List completeTasks = Lists.transform( recentlyFinishedTasks, @@ -687,7 +664,7 @@ public ResourceAction apply(TaskRunnerWorkItem input) if (!optionalTask.isPresent()) { throw new WebApplicationException( Response.serverError().entity( - String.format("No task information found for task with id: [%s]", taskId) + StringUtils.format("No task information found for task with id: [%s]", taskId) ).build() ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java index 2fec23da3fb2..d8cdbf12eec9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java @@ -61,58 +61,56 @@ public SupervisorResourceFilter( @Override public ContainerRequest filter(ContainerRequest request) { - if (getAuthConfig().isEnabled()) { - final String supervisorId = Preconditions.checkNotNull( - request.getPathSegments() - .get( - Iterables.indexOf( - request.getPathSegments(), - new Predicate() + final String supervisorId = Preconditions.checkNotNull( + request.getPathSegments() + .get( + Iterables.indexOf( + request.getPathSegments(), + new Predicate() + { + @Override + public boolean apply(PathSegment input) { - @Override - public boolean apply(PathSegment input) - { - return input.getPath().equals("supervisor"); - } + return input.getPath().equals("supervisor"); } - ) + 1 - ).getPath() + } + ) + 1 + ).getPath() + ); + + Optional supervisorSpecOptional = supervisorManager.getSupervisorSpec(supervisorId); + if (!supervisorSpecOptional.isPresent()) { + throw new WebApplicationException( + Response.status(Response.Status.BAD_REQUEST) + .entity(StringUtils.format("Cannot find any supervisor with id: [%s]", supervisorId)) + .build() ); + } - Optional supervisorSpecOptional = supervisorManager.getSupervisorSpec(supervisorId); - if (!supervisorSpecOptional.isPresent()) { - throw new WebApplicationException( - Response.status(Response.Status.BAD_REQUEST) - .entity(StringUtils.format("Cannot find any supervisor with id: [%s]", supervisorId)) - .build() - ); - } + final SupervisorSpec spec = supervisorSpecOptional.get(); + Preconditions.checkArgument( + spec.getDataSources() != null && spec.getDataSources().size() > 0, + "No dataSources found to perform authorization checks" + ); - final SupervisorSpec spec = supervisorSpecOptional.get(); - Preconditions.checkArgument( - spec.getDataSources() != null && spec.getDataSources().size() > 0, - "No dataSources found to perform authorization checks" - ); + Function resourceActionFunction = getAction(request) == Action.READ ? + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR : + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR; - Function resourceActionFunction = getAction(request) == Action.READ ? - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR : - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR; + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + getReq(), + spec.getDataSources(), + resourceActionFunction, + getAuthorizerMapper() + ); - Access authResult = AuthorizationUtils.authorizeAllResourceActions( - getReq(), - spec.getDataSources(), - resourceActionFunction, - getAuthorizerMapper() - ); - - if (!authResult.isAllowed()) { - throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) - .entity( - String.format("Access-Check-Result: %s", authResult.toString()) - ) - .build()); - } + if (!authResult.isAllowed()) { + throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) + .entity( + StringUtils.format("Access-Check-Result: %s", authResult.toString()) + ) + .build()); } return request; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java index ac41eb0cc069..e70d3006af42 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java @@ -67,52 +67,50 @@ public TaskResourceFilter( @Override public ContainerRequest filter(ContainerRequest request) { - if (getAuthConfig().isEnabled()) { - final String taskId = Preconditions.checkNotNull( - request.getPathSegments() - .get( - Iterables.indexOf( - request.getPathSegments(), - new Predicate() + final String taskId = Preconditions.checkNotNull( + request.getPathSegments() + .get( + Iterables.indexOf( + request.getPathSegments(), + new Predicate() + { + @Override + public boolean apply(PathSegment input) { - @Override - public boolean apply(PathSegment input) - { - return input.getPath().equals("task"); - } + return input.getPath().equals("task"); } - ) + 1 - ).getPath() - ); - - Optional taskOptional = taskStorageQueryAdapter.getTask(taskId); - if (!taskOptional.isPresent()) { - throw new WebApplicationException( - Response.status(Response.Status.BAD_REQUEST) - .entity(StringUtils.format("Cannot find any task with id: [%s]", taskId)) - .build() - ); - } - final String dataSourceName = Preconditions.checkNotNull(taskOptional.get().getDataSource()); + } + ) + 1 + ).getPath() + ); - final ResourceAction resourceAction = new ResourceAction( - new Resource(dataSourceName, ResourceType.DATASOURCE), - getAction(request) + Optional taskOptional = taskStorageQueryAdapter.getTask(taskId); + if (!taskOptional.isPresent()) { + throw new WebApplicationException( + Response.status(Response.Status.BAD_REQUEST) + .entity(StringUtils.format("Cannot find any task with id: [%s]", taskId)) + .build() ); + } + final String dataSourceName = Preconditions.checkNotNull(taskOptional.get().getDataSource()); - final Access authResult = AuthorizationUtils.authorizeResourceAction( - getReq(), - resourceAction, - getAuthorizerMapper() - ); + final ResourceAction resourceAction = new ResourceAction( + new Resource(dataSourceName, ResourceType.DATASOURCE), + getAction(request) + ); - if (!authResult.isAllowed()) { - throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) - .entity( - StringUtils.format("Access-Check-Result: %s", authResult.toString()) - ) - .build()); - } + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizerMapper() + ); + + if (!authResult.isAllowed()) { + throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) + .entity( + StringUtils.format("Access-Check-Result: %s", authResult.toString()) + ) + .build()); } return request; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java index c7c69c242d4d..6dac0cf0c0c4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -85,23 +85,22 @@ public Response specPost(final SupervisorSpec spec, @Context final HttpServletRe @Override public Response apply(SupervisorManager manager) { - if (authConfig.isEnabled()) { - Preconditions.checkArgument( - spec.getDataSources() != null && spec.getDataSources().size() > 0, - "No dataSources found to perform authorization checks" - ); + Preconditions.checkArgument( + spec.getDataSources() != null && spec.getDataSources().size() > 0, + "No dataSources found to perform authorization checks" + ); - Access authResult = AuthorizationUtils.authorizeAllResourceActions( - req, - spec.getDataSources(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizerMapper - ); + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + spec.getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizerMapper + ); - if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); - } + if (!authResult.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); } + manager.createOrUpdateAndStartSupervisor(spec); return Response.ok(ImmutableMap.of("id", spec.getId())).build(); } @@ -120,33 +119,30 @@ public Response specGetAll(@Context final HttpServletRequest req) public Response apply(final SupervisorManager manager) { final Set supervisorIds; - if (authConfig.isEnabled()) { - supervisorIds = Sets.newHashSet(); - for (String supervisorId : manager.getSupervisorIds()) { - Optional supervisorSpecOptional = manager.getSupervisorSpec(supervisorId); - if (supervisorSpecOptional.isPresent()) { - Access accessResult = AuthorizationUtils.authorizeAllResourceActions( - req, - supervisorSpecOptional.get().getDataSources(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizerMapper - ); + supervisorIds = Sets.newHashSet(); + for (String supervisorId : manager.getSupervisorIds()) { + Optional supervisorSpecOptional = manager.getSupervisorSpec(supervisorId); + if (supervisorSpecOptional.isPresent()) { + Access accessResult = AuthorizationUtils.authorizeAllResourceActions( + req, + supervisorSpecOptional.get().getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizerMapper + ); - if (accessResult.isAllowed()) { - supervisorIds.add(supervisorId); - } + if (accessResult.isAllowed()) { + supervisorIds.add(supervisorId); } } - - AuthorizationUtils.authorizeAllResourceActions( - req, - Lists.newArrayList(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizerMapper - ); - } else { - supervisorIds = manager.getSupervisorIds(); } + + AuthorizationUtils.authorizeAllResourceActions( + req, + Lists.newArrayList(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizerMapper + ); + return Response.ok(supervisorIds).build(); } } @@ -239,31 +235,27 @@ public Response specGetAllHistory(@Context final HttpServletRequest req) public Response apply(final SupervisorManager manager) { final Map> supervisorHistory; - if (authConfig.isEnabled()) { - supervisorHistory = Maps.filterKeys( - manager.getSupervisorHistory(), - new Predicate() + supervisorHistory = Maps.filterKeys( + manager.getSupervisorHistory(), + new Predicate() + { + @Override + public boolean apply(String id) { - @Override - public boolean apply(String id) - { - Optional supervisorSpecOptional = manager.getSupervisorSpec(id); - if (!supervisorSpecOptional.isPresent()) { - return false; - } - Access accessResult = AuthorizationUtils.authorizeAllResourceActions( - req, - supervisorSpecOptional.get().getDataSources(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizerMapper - ); - return accessResult.isAllowed(); + Optional supervisorSpecOptional = manager.getSupervisorSpec(id); + if (!supervisorSpecOptional.isPresent()) { + return false; } + Access accessResult = AuthorizationUtils.authorizeAllResourceActions( + req, + supervisorSpecOptional.get().getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizerMapper + ); + return accessResult.isAllowed(); } - ); - } else { - supervisorHistory = manager.getSupervisorHistory(); - } + } + ); return Response.ok(supervisorHistory).build(); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index beca5c0028d7..882f4125c3f9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -39,6 +39,7 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; @@ -75,12 +76,12 @@ public void setUp() throws Exception AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { return new Authorizer() { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { if (resource.getName().equals("allow")) { return new Access(true); @@ -89,11 +90,6 @@ public Access authorize(String identity, Resource resource, Action action) } } - @Override - public String getNamespace() - { - return null; - } }; } }; @@ -111,14 +107,17 @@ public String getNamespace() public void expectAuthorizationTokenCheck() { - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid"); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid"); + + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); EasyMock.expectLastCall().anyTimes(); - req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().anyTimes(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index cc746ed7e56f..c881e0197fd7 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -58,7 +58,11 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryOneTime; @@ -125,7 +129,12 @@ private void tearDownServerAndCurator() @Before public void setUp() throws Exception { - req = EasyMock.createStrictMock(HttpServletRequest.class); + req = EasyMock.createMock(HttpServletRequest.class); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); supervisorManager = EasyMock.createMock(SupervisorManager.class); taskLockbox = EasyMock.createStrictMock(TaskLockbox.class); taskLockbox.syncFromStorage(); @@ -144,7 +153,7 @@ public void setUp() throws Exception taskActionClientFactory = EasyMock.createStrictMock(TaskActionClientFactory.class); EasyMock.expect(taskActionClientFactory.create(EasyMock.anyObject())) .andReturn(null).anyTimes(); - EasyMock.replay(taskLockbox, taskActionClientFactory); + EasyMock.replay(taskLockbox, taskActionClientFactory, req); taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null)); runTaskCountDownLatches = new CountDownLatch[2]; @@ -204,6 +213,16 @@ public void testOverlordRun() throws Exception Thread.sleep(10); } Assert.assertEquals(taskMaster.getCurrentLeader(), druidNode.getHostAndPort()); + + AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { + + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; + // Test Overlord resource stuff overlordResource = new OverlordResource( taskMaster, @@ -212,7 +231,7 @@ public void testOverlordRun() throws Exception null, null, new AuthConfig(), - null + authorizerMapper ); Response response = overlordResource.getLeader(); Assert.assertEquals(druidNode.getHostAndPort(), response.getEntity()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 154e47cd8b33..0411d216ea66 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -23,9 +23,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.TaskMaster; +import io.druid.server.security.AllowAllAuthorizer; +import io.druid.server.security.AuthenticationResult; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.java.util.common.DateTimes; import io.druid.server.security.AuthConfig; import org.easymock.Capture; @@ -61,16 +66,36 @@ public class SupervisorResourceTest extends EasyMockSupport @Before public void setUp() throws Exception { - supervisorResource = new SupervisorResource(taskMaster, new AuthConfig(), null); + AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { + + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; + supervisorResource = new SupervisorResource(taskMaster, new AuthConfig(), authorizerMapper); } @Test public void testSpecPost() throws Exception { - SupervisorSpec spec = new TestSupervisorSpec("my-id", null); + SupervisorSpec spec = new TestSupervisorSpec("my-id", null) { + + @Override + public List getDataSources() + { + return Lists.newArrayList("datasource1"); + } + }; EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)); EasyMock.expect(supervisorManager.createOrUpdateAndStartSupervisor(spec)).andReturn(true); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); replayAll(); Response response = supervisorResource.specPost(spec, request); @@ -93,9 +118,32 @@ public void testSpecPost() throws Exception public void testSpecGetAll() throws Exception { Set supervisorIds = ImmutableSet.of("id1", "id2"); + SupervisorSpec spec1 = new TestSupervisorSpec("id1", null) { + + @Override + public List getDataSources() + { + return Lists.newArrayList("datasource1"); + } + }; + SupervisorSpec spec2 = new TestSupervisorSpec("id2", null) { + + @Override + public List getDataSources() + { + return Lists.newArrayList("datasource2"); + } + }; EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)); EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds); + EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)); + EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); replayAll(); Response response = supervisorResource.specGetAll(request); @@ -221,6 +269,29 @@ public void testSpecGetAllHistory() throws Exception EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).times(2); EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history); + SupervisorSpec spec1 = new TestSupervisorSpec("id1", null) { + + @Override + public List getDataSources() + { + return Lists.newArrayList("datasource1"); + } + }; + SupervisorSpec spec2 = new TestSupervisorSpec("id2", null) { + + @Override + public List getDataSources() + { + return Lists.newArrayList("datasource2"); + } + }; + EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).atLeastOnce(); + EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).atLeastOnce(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); replayAll(); Response response = supervisorResource.specGetAllHistory(request); diff --git a/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java b/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java index 6a2e7f81b317..15a91f35debb 100644 --- a/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java +++ b/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java @@ -29,7 +29,7 @@ import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; import io.druid.server.security.Authenticator; -import io.druid.server.security.NoopAuthenticator; +import io.druid.server.security.AllowAllAuthenticator; public class AuthenticatorModule implements Module { @@ -40,14 +40,14 @@ public void configure(Binder binder) binder, Key.get(Authenticator.class) ); - authenticatorMapBinder.addBinding("noop").to(NoopAuthenticator.class).in(LazySingleton.class); + authenticatorMapBinder.addBinding("allowAll").to(AllowAllAuthenticator.class).in(LazySingleton.class); } @Provides @ManageLifecycle - @Named("noop") + @Named("allowAll") public Authenticator getAuthenticator() { - return new NoopAuthenticator(); + return new AllowAllAuthenticator(); } } diff --git a/server/src/main/java/io/druid/guice/security/AuthorizerModule.java b/server/src/main/java/io/druid/guice/security/AuthorizerModule.java index e711bda1b19c..4de3eb9958ba 100644 --- a/server/src/main/java/io/druid/guice/security/AuthorizerModule.java +++ b/server/src/main/java/io/druid/guice/security/AuthorizerModule.java @@ -29,7 +29,7 @@ import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; import io.druid.server.security.Authorizer; -import io.druid.server.security.NoopAuthorizer; +import io.druid.server.security.AllowAllAuthorizer; public class AuthorizerModule implements Module { @@ -40,14 +40,14 @@ public void configure(Binder binder) binder, Key.get(Authorizer.class) ); - authorizerMapBinder.addBinding("noop").to(NoopAuthorizer.class).in(LazySingleton.class); + authorizerMapBinder.addBinding("allowAll").to(AllowAllAuthorizer.class).in(LazySingleton.class); } @Provides @ManageLifecycle - @Named("noop") + @Named("allowAll") public Authorizer getAuthorizer() { - return new NoopAuthorizer(); + return new AllowAllAuthorizer(); } } diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index c51a09d7942a..2850127e3a92 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -63,6 +63,7 @@ import io.druid.java.util.common.logger.Logger; import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; import io.druid.server.initialization.AuthenticatorHttpClientWrapperModule; +import io.druid.server.initialization.AuthenticatorMapperModule; import io.druid.server.initialization.AuthorizerMapperModule; import io.druid.server.initialization.EmitterModule; import io.druid.server.initialization.jetty.JettyServerModule; @@ -348,10 +349,6 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter // New modules should be added after Log4jShutterDownerModule new Log4jShutterDownerModule(), new DruidAuthModule(), - new AuthenticatorModule(), - new AuthenticatorHttpClientWrapperModule(), - new AuthorizerModule(), - new AuthorizerMapperModule(), new LifecycleModule(), EmitterModule.class, HttpClientModule.global(), @@ -376,6 +373,11 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new FirehoseModule(), new ParsersModule(), new JavaScriptModule(), + new AuthenticatorModule(), + new AuthenticatorMapperModule(), + new AuthenticatorHttpClientWrapperModule(), + new AuthorizerModule(), + new AuthorizerMapperModule(), new StartupLoggingModule() ); diff --git a/server/src/main/java/io/druid/server/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java index 6898c0ec79b5..ff46da0be0de 100644 --- a/server/src/main/java/io/druid/server/ClientInfoResource.java +++ b/server/src/main/java/io/druid/server/ClientInfoResource.java @@ -32,14 +32,9 @@ import io.druid.client.ServerViewUtil; import io.druid.client.TimelineServerView; import io.druid.client.selector.ServerSelector; -<<<<<<< HEAD -import io.druid.common.utils.JodaUtils; -======= import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.JodaUtils; -import io.druid.java.util.common.Pair; ->>>>>>> upstream/master import io.druid.java.util.common.logger.Logger; import io.druid.query.LocatedSegmentDescriptor; import io.druid.query.TableDataSource; @@ -124,16 +119,12 @@ private Map> getSegmentsForDatasources() @Produces(MediaType.APPLICATION_JSON) public Iterable getDataSources(@Context final HttpServletRequest request) { - if (authConfig.isEnabled()) { - return AuthorizationUtils.filterAuthorizedResources( - request, - getSegmentsForDatasources().keySet(), - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizerMapper - ); - } else { - return getSegmentsForDatasources().keySet(); - } + return AuthorizationUtils.filterAuthorizedResources( + request, + getSegmentsForDatasources().keySet(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizerMapper + ); } @GET diff --git a/server/src/main/java/io/druid/server/QueryLifecycle.java b/server/src/main/java/io/druid/server/QueryLifecycle.java index 791e567eff97..142a96a48ece 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycle.java +++ b/server/src/main/java/io/druid/server/QueryLifecycle.java @@ -40,16 +40,9 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; -<<<<<<< HEAD +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; -import org.joda.time.DateTime; -======= -import io.druid.server.security.AuthorizationInfo; -import io.druid.server.security.Resource; -import io.druid.server.security.ResourceType; ->>>>>>> upstream/master import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; @@ -81,7 +74,6 @@ public class QueryLifecycle private final ServiceEmitter emitter; private final RequestLogger requestLogger; private final ServerConfig serverConfig; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; private final long startMs; private final long startNs; @@ -97,7 +89,6 @@ public QueryLifecycle( final ServiceEmitter emitter, final RequestLogger requestLogger, final ServerConfig serverConfig, - final AuthConfig authConfig, final AuthorizerMapper authorizerMapper, final long startMs, final long startNs @@ -109,7 +100,6 @@ public QueryLifecycle( this.emitter = emitter; this.requestLogger = requestLogger; this.serverConfig = serverConfig; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; this.startMs = startMs; this.startNs = startNs; @@ -133,10 +123,8 @@ public QueryLifecycle( @SuppressWarnings("unchecked") public Sequence runSimple( final Query query, - @Nullable final String user, - @Nullable final String namespace, - @Nullable final String remoteAddress, - boolean needsAuth + @Nullable final AuthenticationResult authenticationResult, + @Nullable final String remoteAddress ) { initialize(query); @@ -144,13 +132,9 @@ public Sequence runSimple( final Sequence results; try { - if (needsAuth) { - final Access access = authorize(user, namespace, null); - if (!access.isAllowed()) { - throw new ISE("Unauthorized"); - } - } else { - transition(State.INITIALIZED, State.AUTHORIZED); + final Access access = authorize(authenticationResult, null); + if (!access.isAllowed()) { + throw new ISE("Unauthorized"); } final QueryLifecycle.QueryResponse queryResponse = execute(); @@ -210,43 +194,35 @@ public void initialize(final Query baseQuery) * * */ public Access authorize( - @Nullable final String token, - @Nullable final String namespace, + @Nullable final AuthenticationResult authenticationResult, @Nullable HttpServletRequest req ) { transition(State.INITIALIZED, State.AUTHORIZING); + Access authResult; + if (req != null) { + authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + queryPlus.getQuery().getDataSource().getNames(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizerMapper + ); + } else { + authResult = AuthorizationUtils.authorizeAllResourceActions( + queryPlus.getQuery().getDataSource().getNames(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authenticationResult, + authorizerMapper + ); + } - if (authConfig.isEnabled()) { - Access authResult; - if (req != null) { - authResult = AuthorizationUtils.authorizeAllResourceActions( - req, - queryPlus.getQuery().getDataSource().getNames(), - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizerMapper - ); - } else { - authResult = AuthorizationUtils.authorizeAllResourceActions( - queryPlus.getQuery().getDataSource().getNames(), - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - token, - namespace, - authorizerMapper - ); - } - - if (!authResult.isAllowed()) { - // Not authorized; go straight to Jail, do not pass Go. - transition(State.AUTHORIZING, State.DONE); - } else { - transition(State.AUTHORIZING, State.AUTHORIZED); - } - return authResult; + if (!authResult.isAllowed()) { + // Not authorized; go straight to Jail, do not pass Go. + transition(State.AUTHORIZING, State.DONE); } else { transition(State.AUTHORIZING, State.AUTHORIZED); - return new Access(true); } + return authResult; } /** diff --git a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java index eb698e838fc0..745d23bb5c9e 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycleFactory.java +++ b/server/src/main/java/io/druid/server/QueryLifecycleFactory.java @@ -39,7 +39,6 @@ public class QueryLifecycleFactory private final ServiceEmitter emitter; private final RequestLogger requestLogger; private final ServerConfig serverConfig; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; @Inject @@ -60,7 +59,6 @@ public QueryLifecycleFactory( this.emitter = emitter; this.requestLogger = requestLogger; this.serverConfig = serverConfig; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; } @@ -73,7 +71,6 @@ public QueryLifecycle factorize() emitter, requestLogger, serverConfig, - authConfig, authorizerMapper, System.currentTimeMillis(), System.nanoTime() diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 7bf1271dec09..2879c8ae0085 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -45,6 +45,7 @@ import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import org.joda.time.DateTime; @@ -128,24 +129,23 @@ public Response getServer(@PathParam("id") String queryId, @Context final HttpSe if (log.isDebugEnabled()) { log.debug("Received cancel request for query [%s]", queryId); } - if (authConfig.isEnabled()) { - Set datasources = queryManager.getQueryDatasources(queryId); - if (datasources == null) { - log.warn("QueryId [%s] not registered with QueryManager, cannot cancel", queryId); - datasources = Sets.newTreeSet(); - } + Set datasources = queryManager.getQueryDatasources(queryId); + if (datasources == null) { + log.warn("QueryId [%s] not registered with QueryManager, cannot cancel", queryId); + datasources = Sets.newTreeSet(); + } - Access authResult = AuthorizationUtils.authorizeAllResourceActions( - req, - datasources, - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizerMapper - ); + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + datasources, + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + authorizerMapper + ); - if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); - } + if (!authResult.isAllowed()) { + return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); } + queryManager.cancelQuery(queryId); return Response.status(Response.Status.ACCEPTED).build(); } @@ -177,8 +177,7 @@ public Response doPost( } final Access authResult = queryLifecycle.authorize( - (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), - (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE), + (AuthenticationResult) req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT), req ); if (!authResult.isAllowed()) { diff --git a/server/src/main/java/io/druid/server/http/DatasourcesResource.java b/server/src/main/java/io/druid/server/http/DatasourcesResource.java index dc5a265698db..93d9393a8008 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -44,6 +44,7 @@ import io.druid.query.TableDataSource; import io.druid.server.http.security.DatasourceResourceFilter; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; @@ -109,14 +110,11 @@ public Response getQueryableDataSources( ) { Response.ResponseBuilder builder = Response.ok(); - final Set datasources = authConfig.isEnabled() ? - InventoryViewUtils.getSecuredDataSources( - serverInventoryView, - authorizerMapper, - (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), - (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) - ) : - InventoryViewUtils.getDataSources(serverInventoryView); + final Set datasources = InventoryViewUtils.getSecuredDataSources( + serverInventoryView, + authorizerMapper, + (AuthenticationResult) req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) + ); if (full != null) { return builder.entity(datasources).build(); diff --git a/server/src/main/java/io/druid/server/http/IntervalsResource.java b/server/src/main/java/io/druid/server/http/IntervalsResource.java index 21be08779a49..4a90237e6716 100644 --- a/server/src/main/java/io/druid/server/http/IntervalsResource.java +++ b/server/src/main/java/io/druid/server/http/IntervalsResource.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.MapUtils; import io.druid.java.util.common.guava.Comparators; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -70,14 +71,11 @@ public IntervalsResource( public Response getIntervals(@Context final HttpServletRequest req) { final Comparator comparator = Comparators.inverse(Comparators.intervalsByStartThenEnd()); - final Set datasources = authConfig.isEnabled() ? - InventoryViewUtils.getSecuredDataSources( - serverInventoryView, - authorizerMapper, - (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), - (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) - ) : - InventoryViewUtils.getDataSources(serverInventoryView); + final Set datasources = InventoryViewUtils.getSecuredDataSources( + serverInventoryView, + authorizerMapper, + (AuthenticationResult) req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) + ); final Map>> retVal = Maps.newTreeMap(comparator); for (DruidDataSource dataSource : datasources) { @@ -105,14 +103,11 @@ public Response getSpecificIntervals( ) { final Interval theInterval = Intervals.of(interval.replace("_", "/")); - final Set datasources = authConfig.isEnabled() ? - InventoryViewUtils.getSecuredDataSources( - serverInventoryView, - authorizerMapper, - (String) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN), - (String) req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE) - ) : - InventoryViewUtils.getDataSources(serverInventoryView); + final Set datasources = InventoryViewUtils.getSecuredDataSources( + serverInventoryView, + authorizerMapper, + (AuthenticationResult) req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) + ); final Comparator comparator = Comparators.inverse(Comparators.intervalsByStartThenEnd()); diff --git a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java index 50e0ec52e6e4..e6db453e6c31 100644 --- a/server/src/main/java/io/druid/server/http/InventoryViewUtils.java +++ b/server/src/main/java/io/druid/server/http/InventoryViewUtils.java @@ -33,6 +33,7 @@ import io.druid.java.util.common.Pair; import io.druid.server.security.Access; import io.druid.server.security.Action; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; @@ -82,15 +83,14 @@ public Iterable apply(DruidServer input) public static Set getSecuredDataSources( InventoryView inventoryView, final AuthorizerMapper authorizerMapper, - final String identity, - final String namespace + final AuthenticationResult authenticationResult ) { if (authorizerMapper == null) { throw new ISE("No authorization mapper found"); } - final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); if (authorizer == null) { throw new ISE("Invalid to call a secured method with null Authorizer!!"); } else { @@ -109,7 +109,7 @@ public boolean apply(DruidDataSource input) if (resourceAccessMap.containsKey(key)) { return resourceAccessMap.get(key).isAllowed(); } else { - Access access = authorizer.authorize(identity, key.lhs, key.rhs); + Access access = authorizer.authorize(authenticationResult, key.lhs, key.rhs); resourceAccessMap.put(key, access); return access.isAllowed(); } diff --git a/server/src/main/java/io/druid/server/http/MetadataResource.java b/server/src/main/java/io/druid/server/http/MetadataResource.java index 6f6b007dd91e..fc1fadd11ac1 100644 --- a/server/src/main/java/io/druid/server/http/MetadataResource.java +++ b/server/src/main/java/io/druid/server/http/MetadataResource.java @@ -102,20 +102,14 @@ public String apply(DruidDataSource input) ); } - final Set dataSourceNamesPostAuth; - - if (authConfig.isEnabled()) { - List datasourceNamesList = AuthorizationUtils.filterAuthorizedResources( - req, - dataSourceNamesPreAuth, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizerMapper - ); - - dataSourceNamesPostAuth = Sets.newTreeSet(datasourceNamesList); - } else { - dataSourceNamesPostAuth = dataSourceNamesPreAuth; - } + List datasourceNamesList = AuthorizationUtils.filterAuthorizedResources( + req, + dataSourceNamesPreAuth, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizerMapper + ); + + final Set dataSourceNamesPostAuth = Sets.newTreeSet(datasourceNamesList); // Cannot do both includeDisabled and full, let includeDisabled take priority // Always use dataSourceNamesPostAuth to determine the set of returned dataSources diff --git a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java index 3091e443ae45..078b4ab5072b 100644 --- a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java @@ -56,26 +56,25 @@ public ConfigResourceFilter( @Override public ContainerRequest filter(ContainerRequest request) { - if (getAuthConfig().isEnabled()) { - final ResourceAction resourceAction = new ResourceAction( - new Resource("CONFIG", ResourceType.CONFIG), - getAction(request) - ); + final ResourceAction resourceAction = new ResourceAction( + new Resource("CONFIG", ResourceType.CONFIG), + getAction(request) + ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( - getReq(), - resourceAction, - getAuthorizerMapper() - ); + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizerMapper() + ); - if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); - } + if (!authResult.isAllowed()) { + throw new WebApplicationException( + Response.status(Response.Status.FORBIDDEN) + .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .build() + ); } + return request; } diff --git a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java index aff026b9f383..c3e85c7ad65e 100644 --- a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java @@ -60,25 +60,23 @@ public DatasourceResourceFilter( @Override public ContainerRequest filter(ContainerRequest request) { - if (getAuthConfig().isEnabled()) { - final ResourceAction resourceAction = new ResourceAction( - new Resource(getRequestDatasourceName(request), ResourceType.DATASOURCE), - getAction(request) - ); + final ResourceAction resourceAction = new ResourceAction( + new Resource(getRequestDatasourceName(request), ResourceType.DATASOURCE), + getAction(request) + ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( - getReq(), - resourceAction, - getAuthorizerMapper() - ); + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizerMapper() + ); - if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); - } + if (!authResult.isAllowed()) { + throw new WebApplicationException( + Response.status(Response.Status.FORBIDDEN) + .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .build() + ); } return request; diff --git a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java index b151535bee3c..a5a3c13bcafc 100644 --- a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java @@ -60,41 +60,39 @@ public RulesResourceFilter( @Override public ContainerRequest filter(ContainerRequest request) { - if (getAuthConfig().isEnabled()) { - final String dataSourceName = request.getPathSegments() - .get( - Iterables.indexOf( - request.getPathSegments(), - new Predicate() + final String dataSourceName = request.getPathSegments() + .get( + Iterables.indexOf( + request.getPathSegments(), + new Predicate() + { + @Override + public boolean apply(PathSegment input) { - @Override - public boolean apply(PathSegment input) - { - return input.getPath().equals("rules"); - } + return input.getPath().equals("rules"); } - ) + 1 - ).getPath(); - Preconditions.checkNotNull(dataSourceName); + } + ) + 1 + ).getPath(); + Preconditions.checkNotNull(dataSourceName); - final ResourceAction resourceAction = new ResourceAction( - new Resource(dataSourceName, ResourceType.DATASOURCE), - getAction(request) - ); + final ResourceAction resourceAction = new ResourceAction( + new Resource(dataSourceName, ResourceType.DATASOURCE), + getAction(request) + ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( - getReq(), - resourceAction, - getAuthorizerMapper() - ); + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizerMapper() + ); - if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); - } + if (!authResult.isAllowed()) { + throw new WebApplicationException( + Response.status(Response.Status.FORBIDDEN) + .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .build() + ); } return request; diff --git a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java index e2074793f39e..db7a99485286 100644 --- a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java @@ -62,25 +62,23 @@ public StateResourceFilter( @Override public ContainerRequest filter(ContainerRequest request) { - if (getAuthConfig().isEnabled()) { - final ResourceAction resourceAction = new ResourceAction( - new Resource("STATE", ResourceType.STATE), - getAction(request) - ); + final ResourceAction resourceAction = new ResourceAction( + new Resource("STATE", ResourceType.STATE), + getAction(request) + ); - final Access authResult = AuthorizationUtils.authorizeResourceAction( - getReq(), - resourceAction, - getAuthorizerMapper() - ); + final Access authResult = AuthorizationUtils.authorizeResourceAction( + getReq(), + resourceAction, + getAuthorizerMapper() + ); - if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); - } + if (!authResult.isAllowed()) { + throw new WebApplicationException( + Response.status(Response.Status.FORBIDDEN) + .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .build() + ); } return request; diff --git a/server/src/main/java/io/druid/server/initialization/NoopAuthorizerModule.java b/server/src/main/java/io/druid/server/initialization/AllowAllAuthorizerModule.java similarity index 86% rename from server/src/main/java/io/druid/server/initialization/NoopAuthorizerModule.java rename to server/src/main/java/io/druid/server/initialization/AllowAllAuthorizerModule.java index 244ce4ee4556..bb8798d3377b 100644 --- a/server/src/main/java/io/druid/server/initialization/NoopAuthorizerModule.java +++ b/server/src/main/java/io/druid/server/initialization/AllowAllAuthorizerModule.java @@ -25,11 +25,11 @@ import com.google.inject.name.Named; import io.druid.guice.ManageLifecycle; import io.druid.server.security.Authorizer; -import io.druid.server.security.NoopAuthorizer; +import io.druid.server.security.AllowAllAuthorizer; -public class NoopAuthorizerModule implements Module +public class AllowAllAuthorizerModule implements Module { - public static final String TYPE = "noop"; + public static final String TYPE = "allowAll"; @Override public void configure(Binder binder) @@ -41,6 +41,6 @@ public void configure(Binder binder) @Named(TYPE) public Authorizer makeAuthorizer() { - return new NoopAuthorizer(); + return new AllowAllAuthorizer(); } } diff --git a/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java index dc5f5057e647..78f88fe8eb91 100644 --- a/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java +++ b/server/src/main/java/io/druid/server/initialization/AuthenticatorHttpClientWrapperModule.java @@ -22,13 +22,12 @@ import com.fasterxml.jackson.databind.Module; import com.google.inject.Binder; import com.google.inject.Inject; -import com.google.inject.Injector; import com.google.inject.Provider; import io.druid.guice.LazySingleton; import io.druid.initialization.DruidModule; import io.druid.java.util.common.logger.Logger; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticatorHttpClientWrapper; +import io.druid.server.security.AuthenticatorMapper; import java.util.Collections; import java.util.List; @@ -53,20 +52,18 @@ public List getJacksonModules() private static class AuthenticatorHttpClientWrapperProvider implements Provider { - private AuthConfig authConfig; - private Injector injector; + private AuthenticatorHttpClientWrapper wrapper; @Inject - public void inject(Injector injector) + public void inject(AuthenticatorMapper authenticatorMapper) { - this.authConfig = injector.getInstance(AuthConfig.class); - this.injector = injector; + this.wrapper = new AuthenticatorHttpClientWrapper(authenticatorMapper.getEscalatingAuthenticator()); } @Override public AuthenticatorHttpClientWrapper get() { - return new AuthenticatorHttpClientWrapper(authConfig, injector); + return wrapper; } } } diff --git a/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java new file mode 100644 index 000000000000..c66da3e8a0d2 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java @@ -0,0 +1,120 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.initialization; + +import com.fasterxml.jackson.databind.Module; +import com.google.common.base.Supplier; +import com.google.common.collect.Maps; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Provider; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.JsonConfigurator; +import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleModule; +import io.druid.initialization.DruidModule; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.AllowAllAuthenticator; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +public class AuthenticatorMapperModule implements DruidModule +{ + private static final String AUTHENTICATOR_PROPERTIES_FORMAT_STRING = "druid.auth.authenticator.%s"; + private static Logger log = new Logger(AuthenticatorMapperModule.class); + + @Override + public void configure(Binder binder) + { + binder.bind(AuthenticatorMapper.class) + .toProvider(new AuthenticatorMapperProvider()) + .in(LazySingleton.class); + + LifecycleModule.register(binder, AuthenticatorMapper.class); + } + + @SuppressWarnings("unchecked") + @Override + public List getJacksonModules() + { + return Collections.EMPTY_LIST; + } + + private static class AuthenticatorMapperProvider implements Provider + { + private AuthConfig authConfig; + private Injector injector; + private Properties props; + private JsonConfigurator configurator; + + @Inject + public void inject(Injector injector, Properties props, JsonConfigurator configurator) + { + this.authConfig = injector.getInstance(AuthConfig.class); + this.injector = injector; + this.props = props; + this.configurator = configurator; + } + + @Override + public AuthenticatorMapper get() + { + // order of the authenticators matters + Map authenticatorMap = Maps.newLinkedHashMap(); + + List authenticators = authConfig.getAuthenticatorChain(); + + // If user didn't configure any Authenticators, use the default which accepts all requests. + if (authenticators == null || authenticators.isEmpty()) { + Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + return new AuthenticatorMapper(defaultMap, "allowAll"); + } + + for (String authenticatorName : authenticators) { + final String authenticatorPropertyBase = StringUtils.format(AUTHENTICATOR_PROPERTIES_FORMAT_STRING, authenticatorName); + final JsonConfigProvider authenticatorProvider = new JsonConfigProvider<>( + authenticatorPropertyBase, + Authenticator.class + ); + + authenticatorProvider.inject(props, configurator); + + Supplier authenticatorSupplier = authenticatorProvider.get(); + if (authenticatorSupplier == null) { + throw new ISE("Could not create authenticator with name: %s", authenticatorName); + } + Authenticator authenticator = authenticatorSupplier.get(); + authenticatorMap.put(authenticatorName, authenticator); + } + + return new AuthenticatorMapper(authenticatorMap, authConfig.getEscalatedAuthenticator()); + } + } +} diff --git a/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java index 04bf800b14f0..86c975c60565 100644 --- a/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java +++ b/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java @@ -20,28 +20,33 @@ package io.druid.server.initialization; import com.fasterxml.jackson.databind.Module; +import com.google.common.base.Supplier; import com.google.common.collect.Maps; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Injector; -import com.google.inject.Key; import com.google.inject.Provider; -import com.google.inject.name.Names; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.JsonConfigurator; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.initialization.DruidModule; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.DefaultAuthorizer; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Properties; public class AuthorizerMapperModule implements DruidModule { + private static final String AUTHORIZER_PROPERTIES_FORMAT_STRING = "druid.auth.authorizer.%s"; private static Logger log = new Logger(AuthorizerMapperModule.class); @Override @@ -65,12 +70,16 @@ private static class AuthorizerMapperProvider implements Provider authorizers = authConfig.getAuthorizers(); - // If user didn't configure any Authorizers, use the default which rejects all requests. + // If user didn't configure any Authorizers, use the default which accepts all requests. if (authorizers == null || authorizers.isEmpty()) { return new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { - return new DefaultAuthorizer(); + return new AllowAllAuthorizer(); } }; } for (String authorizerName : authorizers) { - Authorizer authorizer = injector.getInstance(Key.get( - Authorizer.class, - Names.named(authorizerName) - )); + final String authorizerPropertyBase = StringUtils.format(AUTHORIZER_PROPERTIES_FORMAT_STRING, authorizerName); + final JsonConfigProvider authorizerProvider = new JsonConfigProvider<>( + authorizerPropertyBase, + Authorizer.class + ); - authorizerMap.put(authorizer.getNamespace(), authorizer); + authorizerProvider.inject(props, configurator); + + Supplier authorizerSupplier = authorizerProvider.get(); + if (authorizerSupplier == null) { + throw new ISE("Could not create authorizer with name: %s", authorizerName); + } + Authorizer authorizer = authorizerSupplier.get(); + authorizerMap.put(authorizerName, authorizer); } return new AuthorizerMapper(authorizerMap); diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index 40d327757b1a..d1856bf92f1d 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -146,16 +146,12 @@ public LookupCoordinatorManager( druidNodeDiscoveryProvider, configManager, lookupCoordinatorManagerConfig, -<<<<<<< HEAD new LookupsCommunicator( authenticatorHttpClientWrapper.getEscalatedClient(httpClient), lookupCoordinatorManagerConfig, smileMapper - ) -======= - new LookupsCommunicator(httpClient, lookupCoordinatorManagerConfig, smileMapper), + ), null ->>>>>>> upstream/master ); } diff --git a/server/src/main/java/io/druid/server/security/Access.java b/server/src/main/java/io/druid/server/security/Access.java index 19f6a6115753..c7bc32b9dd6e 100644 --- a/server/src/main/java/io/druid/server/security/Access.java +++ b/server/src/main/java/io/druid/server/security/Access.java @@ -23,6 +23,8 @@ public class Access { + public final static Access OK = new Access(true); + private final boolean allowed; private String message; diff --git a/server/src/main/java/io/druid/server/security/Action.java b/server/src/main/java/io/druid/server/security/Action.java index 68a038c362d3..99f6a7e6bbf5 100644 --- a/server/src/main/java/io/druid/server/security/Action.java +++ b/server/src/main/java/io/druid/server/security/Action.java @@ -20,6 +20,7 @@ package io.druid.server.security; import com.fasterxml.jackson.annotation.JsonCreator; +import io.druid.java.util.common.StringUtils; public enum Action { @@ -32,6 +33,6 @@ public static Action fromString(String name) if (name == null) { return null; } - return valueOf(name.toUpperCase()); + return valueOf(StringUtils.toUpperCase(name)); } } diff --git a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java b/server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java similarity index 80% rename from server/src/main/java/io/druid/server/security/NoopAuthenticator.java rename to server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java index 49fe647bad21..99fec24ef653 100644 --- a/server/src/main/java/io/druid/server/security/NoopAuthenticator.java +++ b/server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java @@ -32,8 +32,13 @@ import java.util.EnumSet; import java.util.Map; -public class NoopAuthenticator implements Authenticator +/** + * Should only be used in conjunction with AllowAllAuthorizer. + */ +public class AllowAllAuthenticator implements Authenticator { + public static final AuthenticationResult ALLOW_ALL_RESULT = new AuthenticationResult("allowAll", "allowAll"); + @Override public Class getFilterClass() { @@ -58,12 +63,6 @@ public EnumSet getDispatcherType() return null; } - @Override - public String getNamespace() - { - return "noop"; - } - @Override public Filter getFilter() { @@ -80,7 +79,7 @@ public void doFilter( ServletRequest request, ServletResponse response, FilterChain chain ) throws IOException, ServletException { - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN, "druid"); + request.setAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT, ALLOW_ALL_RESULT); chain.doFilter(request, response); } @@ -95,13 +94,13 @@ public void destroy() @Override public String getAuthChallengeHeader() { - return "noop"; + return null; } @Override - public boolean authenticateJDBCContext(Map context) + public AuthenticationResult authenticateJDBCContext(Map context) { - return true; + return ALLOW_ALL_RESULT; } @Override @@ -109,4 +108,10 @@ public HttpClient createEscalatedClient(HttpClient baseClient) { return baseClient; } + + @Override + public AuthenticationResult createEscalatedAuthenticationResult() + { + return ALLOW_ALL_RESULT; + } } diff --git a/server/src/main/java/io/druid/server/security/NoopAuthorizer.java b/server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java similarity index 79% rename from server/src/main/java/io/druid/server/security/NoopAuthorizer.java rename to server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java index 4b8ddc53854b..1b14f950d886 100644 --- a/server/src/main/java/io/druid/server/security/NoopAuthorizer.java +++ b/server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java @@ -19,17 +19,12 @@ package io.druid.server.security; -public class NoopAuthorizer implements Authorizer +public class AllowAllAuthorizer implements Authorizer { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { - return new Access(true); + return Access.OK; } - @Override - public String getNamespace() - { - return "noop"; - } } diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index 424f67ee1bd3..b464b8d4edab 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -27,22 +27,14 @@ public class AuthConfig { /** - * Use this String as the attribute name for the request attribute to pass an authentication token - * from the servlet filter to the jersey resource + * HTTP attribute that holds an AuthenticationResult, with info about a successful authentication check. */ - public static final String DRUID_AUTH_TOKEN = "Druid-Auth-Token"; + public static final String DRUID_AUTHENTICATION_RESULT = "Druid-Authentication-Result"; /** * HTTP attribute set when a static method in AuthorizationUtils performs an authorization check on the request. */ - public static final String DRUID_AUTH_TOKEN_CHECKED = "Druid-Auth-Token-Checked"; - - /** - * HTTP attribute that indicates the namespace for a request. Set by Authenticator implementations when - * they successfully authenticate a request. The Authorizer with a matching namespace will be used to - * authorize the request. - */ - public static final String DRUID_AUTH_NAMESPACE = "Druid-Auth-Namespace"; + public static final String DRUID_AUTHORIZATION_CHECKED = "Druid-Auth-Token-Checked"; public AuthConfig() { @@ -53,13 +45,13 @@ public AuthConfig() public AuthConfig( @JsonProperty("enabled") boolean enabled, @JsonProperty("authenticatorChain") List authenticationChain, - @JsonProperty("internalAuthenticator") String internalAuthenticator, + @JsonProperty("escalatedAuthenticator") String escalatedAuthenticator, @JsonProperty("authorizers") List authorizers ) { this.enabled = enabled; this.authenticatorChain = authenticationChain; - this.internalAuthenticator = internalAuthenticator; + this.escalatedAuthenticator = escalatedAuthenticator == null ? "allowAll" : escalatedAuthenticator; this.authorizers = authorizers; } @@ -70,7 +62,7 @@ public AuthConfig( private final List authenticatorChain; @JsonProperty - private final String internalAuthenticator; + private final String escalatedAuthenticator; @JsonProperty private List authorizers; @@ -85,9 +77,9 @@ public List getAuthenticatorChain() return authenticatorChain; } - public String getInternalAuthenticator() + public String getEscalatedAuthenticator() { - return internalAuthenticator; + return escalatedAuthenticator; } public List getAuthorizers() @@ -101,7 +93,7 @@ public String toString() return "AuthConfig{" + "enabled=" + enabled + ", authenticatorChain='" + authenticatorChain + '\'' + - ", internalAuthenticator='" + internalAuthenticator + '\'' + + ", escalatedAuthenticator='" + escalatedAuthenticator + '\'' + ", authorizers='" + authorizers + '\'' + '}'; } @@ -126,9 +118,9 @@ public boolean equals(Object o) : that.getAuthenticatorChain() != null) { return false; } - if (getInternalAuthenticator() != null - ? !getInternalAuthenticator().equals(that.getInternalAuthenticator()) - : that.getInternalAuthenticator() != null) { + if (getEscalatedAuthenticator() != null + ? !getEscalatedAuthenticator().equals(that.getEscalatedAuthenticator()) + : that.getEscalatedAuthenticator() != null) { return false; } return getAuthorizers() != null @@ -142,7 +134,7 @@ public int hashCode() { int result = (isEnabled() ? 1 : 0); result = 31 * result + (getAuthenticatorChain() != null ? getAuthenticatorChain().hashCode() : 0); - result = 31 * result + (getInternalAuthenticator() != null ? getInternalAuthenticator().hashCode() : 0); + result = 31 * result + (getEscalatedAuthenticator() != null ? getEscalatedAuthenticator().hashCode() : 0); result = 31 * result + (getAuthorizers() != null ? getAuthorizers().hashCode() : 0); return result; } diff --git a/server/src/main/java/io/druid/server/security/AuthenticationResult.java b/server/src/main/java/io/druid/server/security/AuthenticationResult.java new file mode 100644 index 000000000000..ac92665dcccd --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthenticationResult.java @@ -0,0 +1,61 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * An AuthenticationResult contains information about a successfully authenticated request. + */ +public class AuthenticationResult +{ + /** + * the identity of the requester + */ + private final String identity; + + /** + * the name of the Authorizer that should handle the authenticated request. + */ + private final String authorizerName; + + @JsonCreator + public AuthenticationResult( + @JsonProperty("identity") final String identity, + @JsonProperty("authorizerName") final String authorizerName + ) + { + this.identity = identity; + this.authorizerName = authorizerName; + } + + @JsonProperty + public String getIdentity() + { + return identity; + } + + @JsonProperty + public String getAuthorizerName() + { + return authorizerName; + } +} diff --git a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java index 86cce524dd16..d83dc1d2ccd3 100644 --- a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java @@ -20,11 +20,6 @@ package io.druid.server.security; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.name.Names; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; @@ -32,22 +27,6 @@ public class AuthenticationUtils { - public static List getAuthenticatorChainFromConfig( - List authenticatorChain, - Injector injector - ) - { - Preconditions.checkNotNull(authenticatorChain, "No authenticator chain defined!"); - List authenticators = Lists.newArrayList(); - for (String authenticatorName : authenticatorChain) { - authenticators.add( - injector.getInstance(Key.get(Authenticator.class, Names.named(authenticatorName))) - ); - } - - return authenticators; - } - public static void addAuthenticationFilterChain( ServletContextHandler root, List authenticators diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index ab23308120cf..49426584d682 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -29,25 +29,20 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "noop", value = NoopAuthenticator.class), + @JsonSubTypes.Type(name = "allowAll", value = AllowAllAuthenticator.class), }) /** * This interface is essentially a ServletFilterHolder with additional requirements on the getFilter() method contract, plus: * - * - A method that returns a WWW-Authenticate challenge header appropriate for the authentication mechanism. + * - A method that returns a WWW-Authenticate challenge header appropriate for the + * authentication mechanism, getAuthChallengeHeader(). * - A method for creating a wrapped HTTP client that can authenticate using the Authenticator's authentication scheme, - * used for internal Druid node communications (e.g., broker -> historical messages) + * used for internal Druid node communications (e.g., broker -> historical messages), createEscalatedClient(). * - A method for authenticating credentials contained in a JDBC connection context, used for authenticating Druid SQL - * requests received via JDBC + * requests received via JDBC, authenticateJDBCContext(). */ public interface Authenticator extends ServletFilterHolder { - /** - * @return The namespace associated with this Authenticator. This will be used for choosing the correct - * Authorizer for authorizing requests that have been authenticated by this Authenticator. - */ - public String getNamespace(); - /** * Create a Filter that performs authentication checks on incoming HTTP requests. *

@@ -90,7 +85,7 @@ public interface Authenticator extends ServletFilterHolder * * @return true if the identity represented by the context is successfully authenticated */ - public boolean authenticateJDBCContext(Map context); + public AuthenticationResult authenticateJDBCContext(Map context); /** * Return a client that sends requests with the format/information necessary to authenticate successfully @@ -104,4 +99,9 @@ public interface Authenticator extends ServletFilterHolder * @return HttpClient that sends requests with the credentials of the internal system user */ public HttpClient createEscalatedClient(HttpClient baseClient); + + /** + * @return an AuthenticationResult representing the identity of the internal system user. + */ + public AuthenticationResult createEscalatedAuthenticationResult(); } diff --git a/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java b/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java index c054c2607547..25d4cb7c3d25 100644 --- a/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java +++ b/server/src/main/java/io/druid/server/security/AuthenticatorHttpClientWrapper.java @@ -19,10 +19,6 @@ package io.druid.server.security; -import com.google.common.base.Preconditions; -import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.name.Names; import com.metamx.http.client.HttpClient; /** @@ -31,40 +27,17 @@ */ public class AuthenticatorHttpClientWrapper { - private final AuthConfig authConfig; - private Authenticator internalAuthenticator; + private Authenticator escalatingAuthenticator; public AuthenticatorHttpClientWrapper( - final AuthConfig authConfig, - final Injector injector + final Authenticator escalatingAuthenticator ) { - this.authConfig = authConfig; - - if (authConfig.isEnabled()) { - Preconditions.checkNotNull( - authConfig.getInternalAuthenticator(), - "Auth is enabled but no internal authenticator is configured." - ); - Preconditions.checkNotNull( - authConfig.getAuthenticatorChain(), - "Auth is enabled but no authenticators have been configured." - ); - - String internalAuthenticatorName = authConfig.getInternalAuthenticator(); - internalAuthenticator = injector.getInstance(Key.get( - Authenticator.class, - Names.named(internalAuthenticatorName) - )); - } + this.escalatingAuthenticator = escalatingAuthenticator; } public HttpClient getEscalatedClient(HttpClient baseClient) { - if (authConfig.isEnabled()) { - return internalAuthenticator.createEscalatedClient(baseClient); - } else { - return baseClient; - } + return escalatingAuthenticator.createEscalatedClient(baseClient); } } diff --git a/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java b/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java new file mode 100644 index 000000000000..59eb74f1dfcd --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java @@ -0,0 +1,80 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import io.druid.guice.ManageLifecycle; +import io.druid.java.util.common.lifecycle.LifecycleStart; + +import java.util.List; +import java.util.Map; + +@ManageLifecycle +public class AuthenticatorMapper +{ + private Map authenticatorMap; + private Authenticator escalatingAuthenticator; + + public AuthenticatorMapper( + Map authenticatorMap, + String escalatingAuthenticatorName + ) + { + this.authenticatorMap = authenticatorMap; + this.escalatingAuthenticator = authenticatorMap.get(escalatingAuthenticatorName); + Preconditions.checkNotNull( + escalatingAuthenticator, + "Could not find escalating authenticator with name: %s", + escalatingAuthenticatorName + ); + } + + public Authenticator getAuthenticator(String namespace) + { + return authenticatorMap.get(namespace); + } + + public Authenticator getEscalatingAuthenticator() + { + return escalatingAuthenticator; + } + + public List getAuthenticatorChain() + { + return Lists.newArrayList(authenticatorMap.values()); + } + + @LifecycleStart + public void start() + { + for (Authenticator authenticator : authenticatorMap.values()) { + //authenticator.start(); + } + } + + @LifecycleStart + public void stop() + { + for (Authenticator authenticator : authenticatorMap.values()) { + //authenticator.stop(); + } + } +} diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index f433a64250b2..6f2d0c5e2616 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -37,14 +37,12 @@ */ public class AuthorizationUtils { - public final static Access ACCESS_OK = new Access(true, "All resource-actions authorized."); - /** * Check a resource-action using the authorization fields from the request. * * Otherwise, if the resource-actions is authorized, return ACCESS_OK. * - * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. * * If this attribute is already set when this function is called, an exception is thrown. * @@ -74,7 +72,7 @@ public static Access authorizeResourceAction( * * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. * - * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. * * If this attribute is already set when this function is called, an exception is thrown. * @param request HTTP request to be authorized @@ -87,19 +85,16 @@ public static Access authorizeAllResourceActions( final AuthorizerMapper authorizerMapper ) { - final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - if (identity == null) { - throw new ISE("Null identity."); - } - - final String namespace = (String) request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE); - if (namespace == null) { - throw new ISE("Null namespace."); + final AuthenticationResult authenticationResult = (AuthenticationResult) request.getAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT + ); + if (authenticationResult == null) { + throw new ISE("Null authentication result"); } - final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); if (authorizer == null) { - throw new ISE("No authorizer found for namespace: [%s].", namespace); + throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); } // this method returns on first failure, so only successful Access results are kept in the cache @@ -110,20 +105,20 @@ public static Access authorizeAllResourceActions( continue; } final Access access = authorizer.authorize( - identity, + authenticationResult, resourceAction.getResource(), resourceAction.getAction() ); if (!access.isAllowed()) { - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); return access; } else { resultCache.add(resourceAction); } } - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); - return ACCESS_OK; + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + return Access.OK; } /** @@ -135,7 +130,7 @@ public static Access authorizeAllResourceActions( * * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. * - * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. * * If this attribute is already set when this fImmutableList.>of(SupervisorManager.class, Authorizer.class)unction is called, an exception is thrown. * @@ -152,19 +147,16 @@ public static Access authorizeAllResourceActions( final AuthorizerMapper authorizerMapper ) { - final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - if (identity == null) { - throw new ISE("Null identity."); - } - - final String namespace = (String) request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE); - if (namespace == null) { - throw new ISE("Null namespace."); + final AuthenticationResult authenticationResult = (AuthenticationResult) request.getAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT + ); + if (authenticationResult == null) { + throw new ISE("Null authentication result"); } - final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); if (authorizer == null) { - throw new ISE("No authorizer found for namespace: [%s].", namespace); + throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); } // this method returns on first failure, so only successful Access results are kept in the cache @@ -176,20 +168,20 @@ public static Access authorizeAllResourceActions( continue; } final Access access = authorizer.authorize( - identity, + authenticationResult, resourceAction.getResource(), resourceAction.getAction() ); if (!access.isAllowed()) { - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); return access; } else { resultCache.add(resourceAction); } } - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); - return ACCESS_OK; + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + return Access.OK; } /** @@ -209,17 +201,17 @@ public static Access authorizeAllResourceActions( public static Access authorizeAllResourceActions( final Collection resources, final Function raGenerator, - final String user, - final String namespace, + final AuthenticationResult authenticationResult, final AuthorizerMapper authorizerMapper ) { - if (user == null || namespace == null) { - throw new ISE("null user or namespace"); + if (authenticationResult == null) { + throw new ISE("null authentication result"); } - final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + + final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); if (authorizer == null) { - throw new ISE("No authorizer found for namespace: [%s].", namespace); + throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); } // this method returns on first failure, so only successful Access results are kept in the cache @@ -231,7 +223,7 @@ public static Access authorizeAllResourceActions( continue; } final Access access = authorizer.authorize( - user, + authenticationResult, resourceAction.getResource(), resourceAction.getAction() ); @@ -242,14 +234,14 @@ public static Access authorizeAllResourceActions( } } - return ACCESS_OK; + return Access.OK; } /** * Filter a list of resource-actions using the request's authorization fields, returning a new list of * resource-actions that were authorized. * - * This function will set the DRUID_AUTH_TOKEN_CHECKED attribute in the request. + * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. * * If this attribute is already set when this function is called, an exception is thrown. * @@ -265,19 +257,16 @@ public static List filterAuthorizedResources( final AuthorizerMapper authorizerMapper ) { - final String identity = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - if (identity == null) { - throw new ISE("Null identity."); - } - - final String namespace = (String) request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE); - if (namespace == null) { - throw new ISE("Null namespace."); + final AuthenticationResult authenticationResult = (AuthenticationResult) request.getAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT + ); + if (authenticationResult == null) { + throw new ISE("Null authentication result"); } - final Authorizer authorizer = authorizerMapper.getAuthorizer(namespace); + final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); if (authorizer == null) { - throw new ISE("No authorizer found for namespace: [%s].", namespace); + throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); } int initialSize = resources.size(); @@ -288,7 +277,7 @@ public static List filterAuthorizedResources( Access access = resultCache.get(resourceAction); if (access == null) { access = authorizer.authorize( - identity, + authenticationResult, resourceAction.getResource(), resourceAction.getAction() ); @@ -299,7 +288,7 @@ public static List filterAuthorizedResources( } } - request.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, (filteredResources.size() > 0 || initialSize == 0)); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, (filteredResources.size() > 0 || initialSize == 0)); return filteredResources; } diff --git a/server/src/main/java/io/druid/server/security/Authorizer.java b/server/src/main/java/io/druid/server/security/Authorizer.java index 2eeaa5e07131..12f64ebd8e3d 100644 --- a/server/src/main/java/io/druid/server/security/Authorizer.java +++ b/server/src/main/java/io/druid/server/security/Authorizer.java @@ -22,10 +22,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultAuthorizer.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultAuthorizer.class), - @JsonSubTypes.Type(name = "noop", value = NoopAuthorizer.class) + @JsonSubTypes.Type(name = "denyAll", value = DenyAllAuthorizer.class), + @JsonSubTypes.Type(name = "allowAll", value = AllowAllAuthorizer.class) }) /** * An Authorizer is responsible for performing authorization checks for resource accesses. @@ -41,22 +41,15 @@ public interface Authorizer { /** - * Check if the entity represented by `identity` in `namespace` is authorized to perform `action` on `resource`. + * Check if the entity represented by {@code identity} is authorized to perform {@code action} on {@code resource}. * - * @param identity The identity of the requester - * @param namespace The namespace of the identity + * @param authenticationResult The authentication result of the request * @param resource The resource to be accessed * @param action The action to perform on the resource * * @return An Access object representing the result of the authorization check. */ - Access authorize(String identity, Resource resource, Action action); - - /** - * @return The namespace associated with this Authorizer. Authenticator implementations will - * put the namespace in request headers. - */ - String getNamespace(); + Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action); /** * Authorizers are registered with an AuthorizerMapper. The AuthorizerMapper is lifecycle managed and will diff --git a/server/src/main/java/io/druid/server/security/AuthorizerMapper.java b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java index e6caf0a85b09..ea0e6ba467ee 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizerMapper.java +++ b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java @@ -36,9 +36,9 @@ public AuthorizerMapper( this.authorizerMap = authorizerMap; } - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { - return authorizerMap.get(namespace); + return authorizerMap.get(name); } @LifecycleStart diff --git a/server/src/main/java/io/druid/server/security/DefaultAuthorizer.java b/server/src/main/java/io/druid/server/security/DenyAllAuthorizer.java similarity index 75% rename from server/src/main/java/io/druid/server/security/DefaultAuthorizer.java rename to server/src/main/java/io/druid/server/security/DenyAllAuthorizer.java index 97bf9e6e4930..66e2e9aca684 100644 --- a/server/src/main/java/io/druid/server/security/DefaultAuthorizer.java +++ b/server/src/main/java/io/druid/server/security/DenyAllAuthorizer.java @@ -19,17 +19,11 @@ package io.druid.server.security; -public class DefaultAuthorizer implements Authorizer +public class DenyAllAuthorizer implements Authorizer { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { - return new Access(false, "Please configure a non-default Authorizer."); - } - - @Override - public String getNamespace() - { - return "default"; + return new Access(false, "The DenyAll Authorizer denies all requests."); } } diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index 9b2e77a24171..8036b6848420 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -77,54 +77,50 @@ public void doFilter( ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain ) throws IOException, ServletException { - if (authConfig.isEnabled()) { - QueryInterruptedException unauthorizedError = new QueryInterruptedException( - QueryInterruptedException.UNAUTHORIZED, - null, - null, - DruidNode.getDefaultHost() - ); - unauthorizedError.setStackTrace(new StackTraceElement[0]); - OutputStream out = servletResponse.getOutputStream(); - - Boolean authInfoChecked = null; - final HttpServletResponse response = (HttpServletResponse) servletResponse; - - // Since this is the last filter in the chain, some previous authentication filter - // should have placed an auth token in the request. - // If not, send an auth challenge. - if (servletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN) == null) { - Set supportedAuthSchemes = Sets.newHashSet(); - for (Authenticator authenticator : authenticators) { - String challengeHeader = authenticator.getAuthChallengeHeader(); - if (challengeHeader != null) { - supportedAuthSchemes.add(challengeHeader); - } - } - for (String authScheme : supportedAuthSchemes) { - response.addHeader("WWW-Authenticate", authScheme); + QueryInterruptedException unauthorizedError = new QueryInterruptedException( + QueryInterruptedException.UNAUTHORIZED, + null, + null, + DruidNode.getDefaultHost() + ); + unauthorizedError.setStackTrace(new StackTraceElement[0]); + OutputStream out = servletResponse.getOutputStream(); + + Boolean authInfoChecked = null; + final HttpServletResponse response = (HttpServletResponse) servletResponse; + + // Since this is the last filter in the chain, some previous authentication filter + // should have placed an authentication result in the request. + // If not, send an authentication challenge. + if (servletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) == null) { + Set supportedAuthSchemes = Sets.newHashSet(); + for (Authenticator authenticator : authenticators) { + String challengeHeader = authenticator.getAuthChallengeHeader(); + if (challengeHeader != null) { + supportedAuthSchemes.add(challengeHeader); } - sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); - out.close(); - return; } - - filterChain.doFilter(servletRequest, servletResponse); - - authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); - if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { - String errorMsg = StringUtils.format( - "Request did not have an authorization check performed: %s", - ((HttpServletRequest) servletRequest).getRequestURI() - ); - // Note: rather than throwing an exception here, it would be nice to blank out the original response - // since the request didn't have any authorization checks performed. However, this breaks proxying - // (e.g. OverlordServletProxy), so this is not implemented for now. - log.error(errorMsg); - throw new ISE(errorMsg); + for (String authScheme : supportedAuthSchemes) { + response.addHeader("WWW-Authenticate", authScheme); } - } else { - filterChain.doFilter(servletRequest, servletResponse); + sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); + out.close(); + return; + } + + filterChain.doFilter(servletRequest, servletResponse); + + authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED); + if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { + String errorMsg = StringUtils.format( + "Request did not have an authorization check performed: %s", + ((HttpServletRequest) servletRequest).getRequestURI() + ); + // Note: rather than throwing an exception here, it would be nice to blank out the original response + // since the request didn't have any authorization checks performed. However, this breaks proxying + // (e.g. OverlordServletProxy), so this is not implemented for now. + log.error(errorMsg); + throw new ISE(errorMsg); } } diff --git a/server/src/main/java/io/druid/server/security/ResourceType.java b/server/src/main/java/io/druid/server/security/ResourceType.java index e3bc1dfadfe0..3d4de3ca065b 100644 --- a/server/src/main/java/io/druid/server/security/ResourceType.java +++ b/server/src/main/java/io/druid/server/security/ResourceType.java @@ -20,6 +20,7 @@ package io.druid.server.security; import com.fasterxml.jackson.annotation.JsonCreator; +import io.druid.java.util.common.StringUtils; public enum ResourceType { @@ -33,6 +34,6 @@ public static ResourceType fromString(String name) if (name == null) { return null; } - return valueOf(name.toUpperCase()); + return valueOf(StringUtils.toUpperCase(name)); } } diff --git a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java index 50e128783c75..e53658197eaa 100644 --- a/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/SecuritySanityCheckFilter.java @@ -75,9 +75,9 @@ public void doFilter( OutputStream out = httpResponse.getOutputStream(); // make sure the original request isn't trying to fake the auth token checks - Boolean authInfoChecked = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED); - String authToken = (String) request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN); - if (authInfoChecked != null || authToken != null) { + Boolean authInfoChecked = (Boolean) request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED); + AuthenticationResult result = (AuthenticationResult) request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT); + if (authInfoChecked != null || result != null) { sendJsonError(httpResponse, Response.SC_FORBIDDEN, unauthorizedMessage, out); out.close(); return; diff --git a/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java b/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java index 712c63b583e8..9980c97c5fdd 100644 --- a/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java +++ b/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java @@ -44,7 +44,7 @@ public void doFilter( ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain ) throws IOException, ServletException { - servletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + servletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); filterChain.doFilter(servletRequest, servletResponse); } diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index 5b0b3b8af3ef..994779e85392 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -43,8 +43,8 @@ import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticatorHttpClientWrapper; +import io.druid.server.security.AllowAllAuthenticator; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -339,7 +339,7 @@ public CallbackAction segmentViewInitialized() EasyMock.createMock(QueryWatcher.class), getSmileMapper(), EasyMock.createMock(HttpClient.class), - new AuthenticatorHttpClientWrapper(new AuthConfig(), null), + new AuthenticatorHttpClientWrapper(new AllowAllAuthenticator()), baseView, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), new NoopServiceEmitter(), diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index b5c964342dbe..dbed7467dfc8 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -53,7 +53,7 @@ import io.druid.server.router.QueryHostFinder; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Authorizer; -import io.druid.server.security.NoopAuthorizer; +import io.druid.server.security.AllowAllAuthorizer; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; @@ -119,9 +119,9 @@ public void configure(Binder binder) new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { - return new NoopAuthorizer(); + return new AllowAllAuthorizer(); } } ); diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index bfa3984383ff..46898437ad15 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -45,6 +45,7 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; @@ -73,6 +74,9 @@ public class QueryResourceTest { private static final QueryToolChestWarehouse warehouse = new MapQueryToolChestWarehouse(ImmutableMap., QueryToolChest>of()); private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid"); + + public static final ServerConfig serverConfig = new ServerConfig() { @Override @@ -171,8 +175,9 @@ public void setup() @Test public void testGoodQuery() throws IOException { - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .anyTimes(); EasyMock.replay(testServletRequest); Response response = queryResource.doPost( @@ -199,26 +204,26 @@ public void testBadQuery() throws IOException @Test public void testSecuredQuery() throws Exception { - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .anyTimes(); - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); - - testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); EasyMock.expectLastCall().times(1); - testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); EasyMock.replay(testServletRequest); AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { return new Authorizer() { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { if (resource.getName().equals("allow")) { return new Access(true); @@ -227,11 +232,6 @@ public Access authorize(String identity, Resource resource, Action action) } } - @Override - public String getNamespace() - { - return null; - } }; } }; @@ -280,23 +280,23 @@ public void testSecuredGetServer() throws Exception final CountDownLatch startAwaitLatch = new CountDownLatch(1); final CountDownLatch cancelledCountDownLatch = new CountDownLatch(1); - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); - - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .anyTimes(); - testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); EasyMock.replay(testServletRequest); AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { return new Authorizer() { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { // READ action corresponds to the query // WRITE corresponds to cancellation of query @@ -320,11 +320,6 @@ public Access authorize(String identity, Resource resource, Action action) } } - @Override - public String getNamespace() - { - return null; - } }; } }; @@ -405,14 +400,14 @@ public void testDenySecuredGetServer() throws Exception final CountDownLatch waitFinishLatch = new CountDownLatch(2); final CountDownLatch startAwaitLatch = new CountDownLatch(1); - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); - - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .anyTimes(); - testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, true); + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); - testServletRequest.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, false); + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); EasyMock.expectLastCall().times(1); EasyMock.replay(testServletRequest); @@ -420,12 +415,12 @@ public void testDenySecuredGetServer() throws Exception AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { return new Authorizer() { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { // READ action corresponds to the query // WRITE corresponds to cancellation of query @@ -443,11 +438,6 @@ public Access authorize(String identity, Resource resource, Action action) } } - @Override - public String getNamespace() - { - return null; - } }; } }; diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java index 875c63269e29..102ebb2c9f01 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java @@ -27,8 +27,8 @@ import io.druid.common.config.JacksonConfigManager; import io.druid.java.util.common.Intervals; import io.druid.server.coordinator.helper.DruidCoordinatorSegmentMerger; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticatorHttpClientWrapper; +import io.druid.server.security.AllowAllAuthenticator; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import org.easymock.EasyMock; @@ -461,7 +461,7 @@ private static List> merge(final Collection segme null, null, null, - new AuthenticatorHttpClientWrapper(new AuthConfig(), null) + new AuthenticatorHttpClientWrapper(new AllowAllAuthenticator()) ) { @Override diff --git a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java index 339ae7cdc02f..178f2d463bd2 100644 --- a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java +++ b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java @@ -29,7 +29,9 @@ import io.druid.server.coordination.ServerType; import io.druid.server.security.Access; import io.druid.server.security.Action; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; @@ -52,6 +54,15 @@ public class DatasourcesResourceTest { + private AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { + + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; + private CoordinatorServerView inventoryView; private DruidServer server; private List listDataSources; @@ -122,8 +133,17 @@ public void testGetFullQueryableDataSources() throws Exception EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); - EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + EasyMock.replay(inventoryView, server, request); + DatasourcesResource datasourcesResource = new DatasourcesResource( + inventoryView, + null, + null, + new AuthConfig(), + authorizerMapper + ); Response response = datasourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); DruidDataSource[] resultantDruidDataSources = new DruidDataSource[result.size()]; @@ -144,14 +164,15 @@ public void testGetFullQueryableDataSources() throws Exception @Test public void testSecuredGetFullQueryableDataSources() throws Exception { - EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); - EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid"); + + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .anyTimes(); EasyMock.expect(server.getDataSources()).andReturn( ImmutableList.of(listDataSources.get(0), listDataSources.get(1)) ).atLeastOnce(); - EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").anyTimes(); - EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("druid").anyTimes(); EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); @@ -159,12 +180,12 @@ public void testSecuredGetFullQueryableDataSources() throws Exception AuthorizerMapper authMapper = new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { return new Authorizer() { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult1, Resource resource, Action action) { if (resource.getName().equals("datasource1")) { return new Access(true); @@ -173,11 +194,6 @@ public Access authorize(String identity, Resource resource, Action action) } } - @Override - public String getNamespace() - { - return null; - } }; } }; @@ -225,9 +241,18 @@ public void testGetSimpleQueryableDataSources() throws Exception EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); - EasyMock.replay(inventoryView, server); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); + EasyMock.replay(inventoryView, server, request); + DatasourcesResource datasourcesResource = new DatasourcesResource( + inventoryView, + null, + null, + new AuthConfig(), + authorizerMapper + ); Response response = datasourcesResource.getQueryableDataSources(null, "simple", request); Assert.assertEquals(200, response.getStatus()); List> results = (List>) response.getEntity(); @@ -361,15 +386,9 @@ public void testGetSegmentDataSourceIntervals() EasyMock.replay(inventoryView); List expectedIntervals = new ArrayList<>(); -<<<<<<< HEAD - expectedIntervals.add(new Interval("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); - expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); -======= expectedIntervals.add(Intervals.of("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig()); ->>>>>>> upstream/master + DatasourcesResource datasourcesResource = new DatasourcesResource(inventoryView, null, null, new AuthConfig(), null); Response response = datasourcesResource.getSegmentDataSourceIntervals("invalidDataSource", null, null); Assert.assertEquals(response.getEntity(), null); diff --git a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java index 7ab3795bb096..fe06602969f7 100644 --- a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java +++ b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java @@ -24,7 +24,11 @@ import io.druid.client.InventoryView; import io.druid.java.util.common.Intervals; import io.druid.server.coordination.ServerType; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -42,6 +46,14 @@ public class IntervalsResourceTest { + private AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; + private InventoryView inventoryView; private DruidServer server; private List dataSegmentList; @@ -106,18 +118,15 @@ public void testGetIntervals() EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); - EasyMock.replay(inventoryView); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + EasyMock.replay(inventoryView, request); List expectedIntervals = new ArrayList<>(); -<<<<<<< HEAD - expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - expectedIntervals.add(new Interval("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); -======= expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); expectedIntervals.add(Intervals.of("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); ->>>>>>> upstream/master + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); Response response = intervalsResource.getIntervals(request); TreeMap>> actualIntervals = (TreeMap) response.getEntity(); @@ -139,16 +148,14 @@ public void testSimpleGetSpecificIntervals() EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); - EasyMock.replay(inventoryView); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + EasyMock.replay(inventoryView, request); List expectedIntervals = new ArrayList<>(); -<<<<<<< HEAD - expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); -======= expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); ->>>>>>> upstream/master + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", "simple", null, request); Map> actualIntervals = (Map) response.getEntity(); @@ -165,16 +172,14 @@ public void testFullGetSpecificIntervals() EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); - EasyMock.replay(inventoryView); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + EasyMock.replay(inventoryView, request); List expectedIntervals = new ArrayList<>(); -<<<<<<< HEAD - expectedIntervals.add(new Interval("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); -======= expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig()); ->>>>>>> upstream/master + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, "full", request); TreeMap>> actualIntervals = (TreeMap) response.getEntity(); @@ -193,9 +198,12 @@ public void testGetSpecificIntervals() EasyMock.expect(inventoryView.getInventory()).andReturn( ImmutableList.of(server) ).atLeastOnce(); - EasyMock.replay(inventoryView); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid") + ).atLeastOnce(); + EasyMock.replay(inventoryView, request); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), null); + IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, null, request); Map actualIntervals = (Map) response.getEntity(); diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index 6b70162c7dc4..31e09d7fa9a3 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -23,8 +23,9 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authenticator; -import io.druid.server.security.NoopAuthenticator; +import io.druid.server.security.AllowAllAuthenticator; import io.druid.server.security.PreResponseAuthorizationCheckFilter; import org.easymock.EasyMock; import org.junit.Rule; @@ -39,7 +40,7 @@ public class PreResponseAuthorizationCheckFilterTest { - private static List authenticators = Lists.newArrayList(new NoopAuthenticator()); + private static List authenticators = Lists.newArrayList(new AllowAllAuthenticator()); private static AuthConfig authConfig = new AuthConfig(true, null, null, null); @Rule @@ -48,14 +49,16 @@ public class PreResponseAuthorizationCheckFilterTest @Test public void testValidRequest() throws Exception { + AuthenticationResult authenticationResult = new AuthenticationResult("so-very-valid", "so-very-valid"); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(authenticationResult).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(true).once(); EasyMock.replay(req, resp, filterChain, outputStream); PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( @@ -76,9 +79,7 @@ public void testAuthenticationFailedRequest() throws Exception ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn(null).once(); - resp.addHeader("WWW-Authenticate", "noop"); - EasyMock.expectLastCall().once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).once(); resp.setStatus(401); EasyMock.expectLastCall().once(); resp.setContentType("application/json"); @@ -102,14 +103,16 @@ public void testMissingAuthorizationCheck() throws Exception expectedException.expect(ISE.class); expectedException.expectMessage("Request did not have an authorization check performed: uri"); + AuthenticationResult authenticationResult = new AuthenticationResult("so-very-valid", "so-very-valid"); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("so-very-valid").once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(authenticationResult).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).once(); EasyMock.expect(resp.getStatus()).andReturn(200).once(); EasyMock.expect(req.getRequestURI()).andReturn("uri").once(); resp.setStatus(403); diff --git a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java index 0e0c378d4850..e6dc9c4466e6 100644 --- a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java +++ b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java @@ -36,6 +36,7 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Resource; @@ -109,10 +110,12 @@ public MultivaluedMap getMatrixParameters() ) ).anyTimes(); EasyMock.expect(request.getMethod()).andReturn(requestMethod).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).anyTimes(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("druid").atLeastOnce(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_NAMESPACE)).andReturn("namespace").atLeastOnce(); - req.setAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED, authCheckResult); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).anyTimes(); + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid"); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(authenticationResult) + .atLeastOnce(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, authCheckResult); EasyMock.expectLastCall().anyTimes(); EasyMock.expect(authorizerMapper.getAuthorizer( EasyMock.anyString() @@ -120,16 +123,11 @@ public MultivaluedMap getMatrixParameters() new Authorizer() { @Override - public Access authorize(String identity, Resource resource, Action action) + public Access authorize(AuthenticationResult authenticationResult1, Resource resource, Action action) { return new Access(authCheckResult); } - @Override - public String getNamespace() - { - return null; - } } ).atLeastOnce(); } diff --git a/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java index 8247560bb1d2..9a8e34e3b72c 100644 --- a/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/SecuritySanityCheckFilterTest.java @@ -21,6 +21,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.SecuritySanityCheckFilter; import org.easymock.EasyMock; import org.junit.Test; @@ -39,8 +40,8 @@ public void testValidRequest() throws Exception HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(null).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn(null).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).once(); filterChain.doFilter(req, resp); EasyMock.expectLastCall().once(); EasyMock.replay(req, filterChain); @@ -57,8 +58,10 @@ public void testInvalidRequest() throws Exception FilterChain filterChain = EasyMock.createStrictMock(FilterChain.class); ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN_CHECKED)).andReturn(true).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN)).andReturn("does-not-belong").once(); + AuthenticationResult authenticationResult = new AuthenticationResult("does-not-belong", "does-not-belong"); + + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(true).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(authenticationResult).once(); EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); resp.setStatus(403); EasyMock.expectLastCall().once(); diff --git a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java index b03092159ea7..7e5b65a66387 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java @@ -45,7 +45,7 @@ import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.NoopAuthorizer; +import io.druid.server.security.AllowAllAuthorizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -82,9 +82,9 @@ public void configure(Binder binder) new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { - return new NoopAuthorizer(); + return new AllowAllAuthorizer(); } } ); diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index d255a7642df5..2c401835afc6 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -43,7 +43,7 @@ import io.druid.server.initialization.jetty.ServletFilterHolder; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.NoopAuthorizer; +import io.druid.server.security.AllowAllAuthorizer; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Server; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -133,9 +133,9 @@ public EnumSet getDispatcherType() new AuthorizerMapper(null) { @Override - public Authorizer getAuthorizer(String namespace) + public Authorizer getAuthorizer(String name) { - return new NoopAuthorizer(); + return new AllowAllAuthorizer(); } } ); diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 76c9539256f3..93805c9ce898 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -43,12 +43,8 @@ import io.druid.java.util.common.StringUtils; import io.druid.query.lookup.LookupsState; import io.druid.server.http.HostAndPortWithScheme; -<<<<<<< HEAD -import io.druid.server.listener.announcer.ListenerDiscoverer; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticatorHttpClientWrapper; -======= ->>>>>>> upstream/master +import io.druid.server.security.AllowAllAuthenticator; import org.easymock.EasyMock; import org.joda.time.Duration; import org.junit.After; @@ -84,8 +80,7 @@ public class LookupCoordinatorManagerTest private final JacksonConfigManager configManager = EasyMock.createStrictMock(JacksonConfigManager.class); private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig = new LookupCoordinatorManagerConfig(); private final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = new AuthenticatorHttpClientWrapper( - new AuthConfig(), - null + new AllowAllAuthenticator() ); private static final String LOOKUP_TIER = "lookup_tier"; @@ -1369,12 +1364,8 @@ public void testLookupDiscoverAll() throws Exception druidNodeDiscoveryProvider, configManager, lookupCoordinatorManagerConfig, -<<<<<<< HEAD - authenticatorHttpClientWrapper -======= EasyMock.createMock(LookupCoordinatorManager.LookupsCommunicator.class), lookupNodeDiscovery ->>>>>>> upstream/master ); manager.start(); @@ -1400,12 +1391,8 @@ public void testDiscoverNodesInTier() throws Exception druidNodeDiscoveryProvider, configManager, lookupCoordinatorManagerConfig, -<<<<<<< HEAD - authenticatorHttpClientWrapper -======= EasyMock.createMock(LookupCoordinatorManager.LookupsCommunicator.class), lookupNodeDiscovery ->>>>>>> upstream/master ); manager.start(); diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index df70f546f545..54ede3e418b2 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -45,12 +45,9 @@ import io.druid.server.DruidNode; import io.druid.server.coordinator.rules.IntervalLoadRule; import io.druid.server.coordinator.rules.Rule; -<<<<<<< HEAD -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticatorHttpClientWrapper; -======= +import io.druid.server.security.AllowAllAuthenticator; import io.druid.server.initialization.ServerConfig; ->>>>>>> upstream/master import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.After; @@ -330,17 +327,13 @@ public TestRuleManager( Supplier config ) { -<<<<<<< HEAD super( httpClient, jsonMapper, config, - selector, - new AuthenticatorHttpClientWrapper(new AuthConfig(), null) + null, + new AuthenticatorHttpClientWrapper(new AllowAllAuthenticator()) ); -======= - super(httpClient, jsonMapper, config, null); ->>>>>>> upstream/master } @Override diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 702b89e438c3..3ae124edd497 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -92,6 +92,7 @@ import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationUtils; import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import io.druid.tasklogs.TaskLogStreamer; import io.druid.tasklogs.TaskLogs; import org.eclipse.jetty.server.Handler; @@ -316,31 +317,25 @@ public void initialize(Server server, Injector injector) final AuthConfig authConfig = injector.getInstance(AuthConfig.class); final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); - List authenticators = null; - if (authConfig.isEnabled()) { - AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); - authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); - AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); - } + List authenticators = null; + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = authenticatorMapper.getAuthenticatorChain(); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); JettyServerInitUtils.addExtensionFilters(root, injector); - if (authConfig.isEnabled()) { - // perform no-op authorization for these static resources - AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); - - // Check that requests were authorized before sending responses - AuthenticationUtils.addPreResponseAuthorizationCheckFilter( - root, - authenticators, - jsonMapper, - authConfig - ); - } + // perform no-op authorization for these static resources + AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); + + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); // /status should not redirect, so add first root.addFilter(GuiceFilter.class, "/status/*", null); diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 3419ebc06066..69fee6876d08 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -35,6 +35,7 @@ import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationUtils; import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -109,30 +110,25 @@ public void initialize(Server server, Injector injector) final AuthConfig authConfig = injector.getInstance(AuthConfig.class); final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); + List authenticators = null; - if (authConfig.isEnabled()) { - AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); - authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); - AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); - } + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = authenticatorMapper.getAuthenticatorChain(); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); JettyServerInitUtils.addExtensionFilters(root, injector); - if (authConfig.isEnabled()) { - // perform no-op authorization for these static resources - AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); - - // Check that requests were authorized before sending responses - AuthenticationUtils.addPreResponseAuthorizationCheckFilter( - root, - authenticators, - jsonMapper, - authConfig - ); - } + // perform no-op authorization for these static resources + AuthenticationUtils.addNoopAuthorizationFilters(root, UNSECURED_PATHS); + + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); // /status should not redirect, so add first root.addFilter(GuiceFilter.class, "/status/*", null); diff --git a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java index 5fdb776ff5e7..3ea6afe7e3c2 100644 --- a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java @@ -30,6 +30,7 @@ import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationUtils; import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -54,27 +55,23 @@ public void initialize(Server server, Injector injector) final AuthConfig authConfig = injector.getInstance(AuthConfig.class); final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); + List authenticators = null; - if (authConfig.isEnabled()) { - AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); - authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); - AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); - } + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = authenticatorMapper.getAuthenticatorChain(); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); + JettyServerInitUtils.addExtensionFilters(root, injector); - if (authConfig.isEnabled()) { - // Check that requests were authorized before sending responses - AuthenticationUtils.addPreResponseAuthorizationCheckFilter( - root, - authenticators, - jsonMapper, - authConfig - ); - } + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); root.addFilter(GuiceFilter.class, "/*", null); diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java index ba6f7f7925a6..fa3f2e39c4d1 100644 --- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -32,6 +32,7 @@ import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationUtils; import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -64,27 +65,22 @@ public void initialize(Server server, Injector injector) final AuthConfig authConfig = injector.getInstance(AuthConfig.class); final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); + List authenticators = null; - if (authConfig.isEnabled()) { - AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); - authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); - AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); - } + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = authenticatorMapper.getAuthenticatorChain(); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); JettyServerInitUtils.addExtensionFilters(root, injector); - if (authConfig.isEnabled()) { - // Check that requests were authorized before sending responses - AuthenticationUtils.addPreResponseAuthorizationCheckFilter( - root, - authenticators, - jsonMapper, - authConfig - ); - } + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); root.addFilter(GuiceFilter.class, "/*", null); diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 87a723da1999..a1f1b41c414f 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -34,6 +34,7 @@ import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationUtils; import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.HandlerList; @@ -83,27 +84,23 @@ public void initialize(Server server, Injector injector) final AuthConfig authConfig = injector.getInstance(AuthConfig.class); final ObjectMapper jsonMapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); + List authenticators = null; - if (authConfig.isEnabled()) { - AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); - authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); - AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); - } + AuthenticationUtils.addSecuritySanityCheckFilter(root, jsonMapper); + authenticators = authenticatorMapper.getAuthenticatorChain(); + AuthenticationUtils.addAuthenticationFilterChain(root, authenticators); JettyServerInitUtils.addExtensionFilters(root, injector); - if (authConfig.isEnabled()) { - // Check that requests were authorized before sending responses - AuthenticationUtils.addPreResponseAuthorizationCheckFilter( - root, - authenticators, - jsonMapper, - authConfig - ); - } + // Check that requests were authorized before sending responses + AuthenticationUtils.addPreResponseAuthorizationCheckFilter( + root, + authenticators, + jsonMapper, + authConfig + ); + // Can't use '/*' here because of Guice conflicts with AsyncQueryForwardingServlet path root.addFilter(GuiceFilter.class, "/status/*", null); diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java index 2eb29ac7ccf6..0af42ba916d1 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java @@ -21,7 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; @@ -41,7 +41,7 @@ public class DruidConnection private final String connectionId; private final int maxStatements; - private final ImmutableMap context; + private final Map context; private final AtomicInteger statementCounter = new AtomicInteger(); private final AtomicReference> timeoutFuture = new AtomicReference<>(); @@ -55,7 +55,7 @@ public DruidConnection(final String connectionId, final int maxStatements, final { this.connectionId = Preconditions.checkNotNull(connectionId); this.maxStatements = maxStatements; - this.context = ImmutableMap.copyOf(context); + this.context = Maps.newHashMap(context); this.statements = new HashMap<>(); } diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java index af236ea8af9c..7f742291b581 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java @@ -29,17 +29,15 @@ import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; -<<<<<<< HEAD import com.google.inject.Injector; -======= import io.druid.java.util.common.DateTimes; ->>>>>>> upstream/master import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthenticationUtils; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerFactory; import org.apache.calcite.avatica.MetaImpl; @@ -97,14 +95,8 @@ public DruidMeta( .build() ); - if (authConfig.isEnabled()) { - this.authenticators = AuthenticationUtils.getAuthenticatorChainFromConfig( - authConfig.getAuthenticatorChain(), - injector - ); - } else { - this.authenticators = null; - } + final AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); + this.authenticators = authenticatorMapper.getAuthenticatorChain(); } @Override @@ -153,16 +145,12 @@ public StatementHandle prepare( final StatementHandle statement = createStatement(ch); final DruidStatement druidStatement = getDruidStatement(statement); final DruidConnection druidConnection = getDruidConnection(statement.connectionId); - String user = null; - String namespace = null; - if (authConfig.isEnabled()) { - if (!authenticateConnection(druidConnection)) { - throw new SecurityException("Authentication failed."); - } - user = getConnectionUser(druidConnection); - namespace = getConnectionNamespace(druidConnection); + AuthenticationResult authenticationResult = null; + if (!authenticateConnection(druidConnection)) { + throw new SecurityException("Authentication failed."); } - statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, user, namespace).getSignature(); + authenticationResult = getConnectionAuthenticationResult(druidConnection); + statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, authenticationResult).getSignature(); return statement; } @@ -191,17 +179,13 @@ public ExecuteResult prepareAndExecute( // Ignore "callback", this class is designed for use with LocalService which doesn't use it. final DruidStatement druidStatement = getDruidStatement(statement); final DruidConnection druidConnection = getDruidConnection(statement.connectionId); - String user = null; - String namespace = null; - if (authConfig != null && authConfig.isEnabled()) { - if (!authenticateConnection(druidConnection)) { - throw new SecurityException("Authentication failed."); - } - ; - user = getConnectionUser(druidConnection); - namespace = getConnectionNamespace(druidConnection); + AuthenticationResult authenticationResult = null; + if (!authenticateConnection(druidConnection)) { + throw new SecurityException("Authentication failed."); } - final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, user, namespace).getSignature(); + authenticationResult = getConnectionAuthenticationResult(druidConnection); + + final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, authenticationResult).getSignature(); final Frame firstFrame = druidStatement.execute() .nextFrame( DruidStatement.START_OFFSET, @@ -532,24 +516,19 @@ private boolean authenticateConnection(final DruidConnection connection) { Map context = connection.context(); for (Authenticator authenticator : authenticators) { - if (authenticator.authenticateJDBCContext(context)) { - context.put("namespace", authenticator.getNamespace()); + AuthenticationResult authenticationResult = authenticator.authenticateJDBCContext(context); + if (authenticationResult != null) { + context.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); return true; } } return false; } - private String getConnectionUser(final DruidConnection connection) - { - Map context = connection.context(); - return (String) context.get("user"); - } - - private String getConnectionNamespace(final DruidConnection connection) + private AuthenticationResult getConnectionAuthenticationResult(final DruidConnection connection) { Map context = connection.context(); - return (String) context.get("namespace"); + return (AuthenticationResult) context.get(AuthConfig.DRUID_AUTHENTICATION_RESULT); } private DruidConnection openDruidConnection(final String connectionId, final Map context) diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java index 47604c083b60..e617e6b8b3a5 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidStatement.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.Yielders; +import io.druid.server.security.AuthenticationResult; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerFactory; import io.druid.sql.calcite.planner.PlannerResult; @@ -156,14 +157,13 @@ public DruidStatement prepare( final PlannerFactory plannerFactory, final String query, final long maxRowCount, - final String user, - final String namespace - ) + final AuthenticationResult authenticationResult + ) { try (final DruidPlanner planner = plannerFactory.createPlanner(queryContext)) { synchronized (lock) { ensure(State.NEW); - this.plannerResult = planner.plan(query, null, user, namespace); + this.plannerResult = planner.plan(query, null, authenticationResult); this.maxRowCount = maxRowCount; this.query = query; this.signature = Meta.Signature.create( diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java index a9d6421a7ad9..6704205855fb 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java @@ -28,6 +28,8 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticationResult; +import io.druid.server.security.AuthenticatorMapper; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; import io.druid.sql.calcite.rel.DruidConvention; @@ -69,30 +71,34 @@ public class DruidPlanner implements Closeable private final PlannerContext plannerContext; private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; + private final AuthenticatorMapper authenticatorMapper; public DruidPlanner( final Planner planner, final PlannerContext plannerContext, final AuthConfig authConfig, - final AuthorizerMapper authorizerMapper + final AuthorizerMapper authorizerMapper, + final AuthenticatorMapper authenticatorMapper ) { this.planner = planner; this.plannerContext = plannerContext; this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; + this.authenticatorMapper = authenticatorMapper; } public PlannerResult plan(final String sql) throws SqlParseException, ValidationException, RelConversionException { - return plan(sql, null, null, null); + AuthenticationResult authenticationResult = authenticatorMapper.getEscalatingAuthenticator() + .createEscalatedAuthenticationResult(); + return plan(sql, null, authenticationResult); } public PlannerResult plan( final String sql, final HttpServletRequest request, - final String user, - final String namespace + final AuthenticationResult authenticationResult ) throws SqlParseException, ValidationException, RelConversionException, SecurityException { SqlExplain explain = null; @@ -105,12 +111,12 @@ public PlannerResult plan( final RelRoot root = planner.rel(validated); try { - return planWithDruidConvention(explain, root, request, user, namespace); + return planWithDruidConvention(explain, root, request, authenticationResult); } catch (RelOptPlanner.CannotPlanException e) { // Try again with BINDABLE convention. Used for querying Values, metadata tables, and fallback. try { - return planWithBindableConvention(explain, root, request, user, namespace); + return planWithBindableConvention(explain, root, request, authenticationResult); } catch (Exception e2) { e.addSuppressed(e2); @@ -134,8 +140,7 @@ private PlannerResult planWithDruidConvention( final SqlExplain explain, final RelRoot root, final HttpServletRequest request, - final String user, - final String namespace + final AuthenticationResult authenticationResult ) throws RelConversionException, SecurityException { final DruidRel druidRel = (DruidRel) planner.transform( @@ -146,29 +151,36 @@ private PlannerResult planWithDruidConvention( root.rel ); - if (authConfig != null && authConfig.isEnabled()) { - List datasourceNames = druidRel.getDatasourceNames(); - Access authResult; - if (request != null) { - authResult = AuthorizationUtils.authorizeAllResourceActions( - request, - datasourceNames, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizerMapper - ); - } else { - authResult = AuthorizationUtils.authorizeAllResourceActions( - datasourceNames, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - user, - namespace, - authorizerMapper - ); - } + List datasourceNames = druidRel.getDatasourceNames(); + // we'll eventually run a second authorization check at QueryLifecycle.runSimple(), so store the + // authentication result in the planner context. + Access authResult; + if (request != null) { + authResult = AuthorizationUtils.authorizeAllResourceActions( + request, + datasourceNames, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authorizerMapper + ); + plannerContext.getQueryContext() + .put( + PlannerContext.CTX_AUTHENTICATION_RESULT, + request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) + ); + } else { + authResult = AuthorizationUtils.authorizeAllResourceActions( + datasourceNames, + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + authenticationResult, + authorizerMapper + ); + plannerContext.getQueryContext().put(PlannerContext.CTX_AUTHENTICATION_RESULT, authenticationResult); + } - if (!authResult.isAllowed()) { - throw new SecurityException(authResult.toString()); - } + + + if (!authResult.isAllowed()) { + throw new SecurityException(authResult.toString()); } if (explain != null) { @@ -205,7 +217,11 @@ public Object[] apply(final Object[] input) } } - private Access authorizeBindableRel(BindableRel rel, HttpServletRequest req, final String user, final String namespace) + private Access authorizeBindableRel( + BindableRel rel, + HttpServletRequest req, + final AuthenticationResult authenticationResult + ) { Set datasourceNames = Sets.newHashSet(); rel.childrenAccept( @@ -238,8 +254,7 @@ public void visit(RelNode node, int ordinal, RelNode parent) return AuthorizationUtils.authorizeAllResourceActions( datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - user, - namespace, + authenticationResult, authorizerMapper ); } @@ -249,8 +264,7 @@ private PlannerResult planWithBindableConvention( final SqlExplain explain, final RelRoot root, final HttpServletRequest request, - final String user, - final String namespace + final AuthenticationResult authenticationResult ) throws RelConversionException { BindableRel bindableRel = (BindableRel) planner.transform( @@ -276,12 +290,10 @@ private PlannerResult planWithBindableConvention( root.validatedRowType ); } - - if (authConfig != null && authConfig.isEnabled()) { - Access accessResult = authorizeBindableRel(bindableRel, request, user, namespace); - if (!accessResult.isAllowed()) { - throw new SecurityException(accessResult.toString()); - } + + Access accessResult = authorizeBindableRel(bindableRel, request, authenticationResult); + if (!accessResult.isAllowed()) { + throw new SecurityException(accessResult.toString()); } if (explain != null) { diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java index cc4b6116a630..d1a5b36cfd99 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java @@ -21,7 +21,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import io.druid.math.expr.ExprMacroTable; +import io.druid.server.security.AuthorizerMapper; import org.apache.calcite.DataContext; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.linq4j.QueryProvider; @@ -40,6 +42,7 @@ public class PlannerContext { public static final String CTX_SQL_CURRENT_TIMESTAMP = "sqlCurrentTimestamp"; public static final String CTX_SQL_TIME_ZONE = "sqlTimeZone"; + public static final String CTX_AUTHENTICATION_RESULT = "authenticationResult"; private final DruidOperatorTable operatorTable; private final ExprMacroTable macroTable; @@ -47,27 +50,31 @@ public class PlannerContext private final DateTime localNow; private final long queryStartTimeMillis; private final Map queryContext; + private final AuthorizerMapper authorizerMapper; private PlannerContext( final DruidOperatorTable operatorTable, final ExprMacroTable macroTable, final PlannerConfig plannerConfig, final DateTime localNow, + final AuthorizerMapper authorizerMapper, final Map queryContext ) { this.operatorTable = operatorTable; this.macroTable = macroTable; this.plannerConfig = Preconditions.checkNotNull(plannerConfig, "plannerConfig"); - this.queryContext = queryContext != null ? ImmutableMap.copyOf(queryContext) : ImmutableMap.of(); + this.queryContext = queryContext != null ? Maps.newHashMap(queryContext) : Maps.newHashMap(); this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.queryStartTimeMillis = System.currentTimeMillis(); + this.authorizerMapper = authorizerMapper; } public static PlannerContext create( final DruidOperatorTable operatorTable, final ExprMacroTable macroTable, final PlannerConfig plannerConfig, + final AuthorizerMapper authorizerMapper, final Map queryContext ) { @@ -99,6 +106,7 @@ public static PlannerContext create( macroTable, plannerConfig.withOverrides(queryContext), utcNow.withZone(timeZone), + authorizerMapper, queryContext ); } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java index 4ce30b35ab19..0076f0bfdb27 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerFactory.java @@ -23,6 +23,7 @@ import io.druid.math.expr.ExprMacroTable; import io.druid.server.QueryLifecycleFactory; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthenticatorMapper; import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.rel.QueryMaker; import io.druid.sql.calcite.schema.DruidSchema; @@ -58,6 +59,7 @@ public class PlannerFactory private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; + private final AuthenticatorMapper authenticatorMapper; @Inject public PlannerFactory( @@ -67,6 +69,7 @@ public PlannerFactory( final ExprMacroTable macroTable, final PlannerConfig plannerConfig, final AuthConfig authConfig, + final AuthenticatorMapper authenticatorMapper, final AuthorizerMapper authorizerMapper ) { @@ -77,12 +80,19 @@ public PlannerFactory( this.plannerConfig = plannerConfig; this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; + this.authenticatorMapper = authenticatorMapper; } public DruidPlanner createPlanner(final Map queryContext) { final SchemaPlus rootSchema = Calcites.createRootSchema(druidSchema); - final PlannerContext plannerContext = PlannerContext.create(operatorTable, macroTable, plannerConfig, queryContext); + final PlannerContext plannerContext = PlannerContext.create( + operatorTable, + macroTable, + plannerConfig, + authorizerMapper, + queryContext + ); final QueryMaker queryMaker = new QueryMaker(queryLifecycleFactory, plannerContext); final FrameworkConfig frameworkConfig = Frameworks .newConfigBuilder() @@ -98,6 +108,12 @@ public DruidPlanner createPlanner(final Map queryContext) .typeSystem(DruidTypeSystem.INSTANCE) .build(); - return new DruidPlanner(Frameworks.getPlanner(frameworkConfig), plannerContext, authConfig, authorizerMapper); + return new DruidPlanner( + Frameworks.getPlanner(frameworkConfig), + plannerContext, + authConfig, + authorizerMapper, + authenticatorMapper + ); } } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index da3b57604e8a..e2e9c84d6572 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -47,6 +47,7 @@ import io.druid.segment.DimensionHandlerUtils; import io.druid.segment.column.Column; import io.druid.server.QueryLifecycleFactory; +import io.druid.server.security.AuthenticationResult; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerContext; import org.apache.calcite.avatica.ColumnMetaData; @@ -231,8 +232,10 @@ private Sequence runQuery(final Query query) { Hook.QUERY_PLAN.run(query); - // Authorization has already been checked during planning, skip authorization checks here. - return queryLifecycleFactory.factorize().runSimple(query, null, null, null, false); + final AuthenticationResult authenticationResult = + (AuthenticationResult) plannerContext.getQueryContext().get(PlannerContext.CTX_AUTHENTICATION_RESULT); + + return queryLifecycleFactory.factorize().runSimple(query, authenticationResult, null); } private Sequence executeTimeseries( diff --git a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java index 6f8b620bb149..a0246920ae4b 100644 --- a/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/io/druid/sql/calcite/schema/DruidSchema.java @@ -50,6 +50,9 @@ import io.druid.segment.column.ValueType; import io.druid.server.QueryLifecycleFactory; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.security.AuthenticationResult; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; import io.druid.sql.calcite.table.RowSignature; @@ -114,6 +117,9 @@ public class DruidSchema extends AbstractSchema // All segments that need to be refreshed. private final TreeSet segmentsNeedingRefresh = new TreeSet<>(SEGMENT_ORDER); + // Escalating authenticator, so we can attach an authentication result to queries we generate. + private final Authenticator escalatingAuthenticator; + private boolean refreshImmediately = false; private long lastRefresh = 0L; private boolean isServerViewInitialized = false; @@ -123,7 +129,8 @@ public DruidSchema( final QueryLifecycleFactory queryLifecycleFactory, final TimelineServerView serverView, final PlannerConfig config, - final ViewManager viewManager + final ViewManager viewManager, + final AuthenticatorMapper authenticatorMapper ) { this.queryLifecycleFactory = Preconditions.checkNotNull(queryLifecycleFactory, "queryLifecycleFactory"); @@ -132,6 +139,7 @@ public DruidSchema( this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager"); this.cacheExec = ScheduledExecutors.fixed(1, "DruidSchema-Cache-%d"); this.tables = Maps.newConcurrentMap(); + this.escalatingAuthenticator = authenticatorMapper.getEscalatingAuthenticator(); serverView.registerTimelineCallback( MoreExecutors.sameThreadExecutor(), @@ -400,7 +408,8 @@ private Set refreshSegmentsForDataSource( final Set retVal = new HashSet<>(); final Sequence sequence = runSegmentMetadataQuery( queryLifecycleFactory, - Iterables.limit(segments, MAX_SEGMENTS_PER_QUERY) + Iterables.limit(segments, MAX_SEGMENTS_PER_QUERY), + escalatingAuthenticator.createEscalatedAuthenticationResult() ); Yielder yielder = Yielders.each(sequence); @@ -470,7 +479,8 @@ private DruidTable buildDruidTable(final String dataSource) private static Sequence runSegmentMetadataQuery( final QueryLifecycleFactory queryLifecycleFactory, - final Iterable segments + final Iterable segments, + final AuthenticationResult authenticationResult ) { // Sanity check: getOnlyElement of a set, to ensure all segments have the same dataSource. @@ -495,8 +505,7 @@ private static Sequence runSegmentMetadataQuery( false ); - // This is an internally generated query, no authorization is needed. - return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, null, null, null, false); + return queryLifecycleFactory.factorize().runSimple(segmentMetadataQuery, authenticationResult, null); } private static RowSignature analysisToRowSignature(final SegmentAnalysis analysis) diff --git a/sql/src/main/java/io/druid/sql/http/SqlResource.java b/sql/src/main/java/io/druid/sql/http/SqlResource.java index c5d361df409b..8cc1d1f4d8d0 100644 --- a/sql/src/main/java/io/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/io/druid/sql/http/SqlResource.java @@ -85,7 +85,7 @@ public Response doPost( final DateTimeZone timeZone; try (final DruidPlanner planner = plannerFactory.createPlanner(sqlQuery.getContext())) { - plannerResult = planner.plan(sqlQuery.getQuery(), req, null, null); + plannerResult = planner.plan(sqlQuery.getQuery(), req, null); timeZone = planner.getPlannerContext().getTimeZone(); // Remember which columns are time-typed, so we can emit ISO8601 instead of millis values. diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index 074a970fc5f4..38ba9418259e 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -30,13 +30,24 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.guice.GuiceInjectors; +import io.druid.initialization.Initialization; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.math.expr.ExprMacroTable; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -73,6 +84,7 @@ import java.sql.Timestamp; import java.sql.Types; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -112,6 +124,7 @@ public int getMaxStatementsPerConnection() private Connection clientLosAngeles; private DruidMeta druidMeta; private String url; + private Injector injector; @Before public void setUp() throws Exception @@ -122,6 +135,26 @@ public void setUp() throws Exception final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); + final Map testAuthorizerMap = new HashMap<>(); + testAuthorizerMap.put("allowAll", new AllowAllAuthorizer()); + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + + injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + } + } + ) + ); druidMeta = new DruidMeta( new PlannerFactory( druidSchema, @@ -130,11 +163,12 @@ public void setUp() throws Exception macroTable, plannerConfig, new AuthConfig(), - new AuthorizerMapper(null) + new AuthenticatorMapper(defaultMap, "allowAll"), + new AuthorizerMapper(testAuthorizerMap) ), AVATICA_CONFIG, new AuthConfig(), - null + injector ); final DruidAvaticaHandler handler = new DruidAvaticaHandler( druidMeta, @@ -563,7 +597,10 @@ public int getMaxRowsPerFrame() final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); - + final Map testAuthorizerMap = new HashMap<>(); + testAuthorizerMap.put("allowAll", new AllowAllAuthorizer()); + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); final List frames = new ArrayList<>(); DruidMeta smallFrameDruidMeta = new DruidMeta( new PlannerFactory( @@ -573,11 +610,12 @@ public int getMaxRowsPerFrame() macroTable, plannerConfig, new AuthConfig(), - new AuthorizerMapper(null) + new AuthenticatorMapper(defaultMap, "allowAll"), + new AuthorizerMapper(testAuthorizerMap) ), smallFrameConfig, new AuthConfig(), - null + injector ) { @Override diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java index e4b60ad27bd5..319d3e281dfc 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java @@ -21,9 +21,16 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import io.druid.java.util.common.DateTimes; import io.druid.math.expr.ExprMacroTable; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -42,6 +49,7 @@ import org.junit.rules.TemporaryFolder; import java.util.List; +import java.util.Map; public class DruidStatementTest { @@ -66,6 +74,8 @@ public void setUp() throws Exception ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); plannerFactory = new PlannerFactory( druidSchema, CalciteTests.createMockQueryLifecycleFactory(walker), @@ -73,7 +83,14 @@ public void setUp() throws Exception macroTable, plannerConfig, new AuthConfig(), - null + new AuthenticatorMapper(defaultMap, "allowAll"), + new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + } ); } @@ -88,7 +105,8 @@ public void tearDown() throws Exception public void testSignature() throws Exception { final String sql = "SELECT * FROM druid.foo"; - final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1, null, null); + final DruidStatement statement = new DruidStatement("", 0, null, () -> { + }).prepare(plannerFactory, sql, -1, AllowAllAuthenticator.ALLOW_ALL_RESULT); // Check signature. final Meta.Signature signature = statement.getSignature(); @@ -127,7 +145,8 @@ public List apply(final ColumnMetaData columnMetaData) public void testSelectAllInFirstFrame() throws Exception { final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo"; - final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1, null, null); + final DruidStatement statement = new DruidStatement("", 0, null, () -> { + }).prepare(plannerFactory, sql, -1, AllowAllAuthenticator.ALLOW_ALL_RESULT); // First frame, ask for all rows. Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 6); @@ -153,7 +172,8 @@ public void testSelectAllInFirstFrame() throws Exception public void testSelectSplitOverTwoFrames() throws Exception { final String sql = "SELECT __time, cnt, dim1, dim2, m1 FROM druid.foo"; - final DruidStatement statement = new DruidStatement("", 0, null, () -> {}).prepare(plannerFactory, sql, -1, null, null); + final DruidStatement statement = new DruidStatement("", 0, null, () -> { + }).prepare(plannerFactory, sql, -1, AllowAllAuthenticator.ALLOW_ALL_RESULT); // First frame, ask for 2 rows. Meta.Frame frame = statement.execute().nextFrame(DruidStatement.START_OFFSET, 2); diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index d030043e6046..78ec6a90a813 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import io.druid.hll.HLLCV1; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; @@ -84,7 +85,13 @@ import io.druid.segment.column.Column; import io.druid.segment.column.ValueType; import io.druid.segment.virtual.ExpressionVirtualColumn; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -175,28 +182,32 @@ public int getMaxQueryCount() private static final Map QUERY_CONTEXT_DEFAULT = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT ); private static final Map QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", "skipEmptyBuckets", false, QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT ); private static final Map QUERY_CONTEXT_NO_TOPN = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, "false", QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT ); private static final Map QUERY_CONTEXT_LOS_ANGELES = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES, QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT ); // Matches QUERY_CONTEXT_DEFAULT @@ -204,17 +215,24 @@ public int getMaxQueryCount() PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", "skipEmptyBuckets", true, QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, + PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT ); // Matches QUERY_CONTEXT_LOS_ANGELES - public static final Map TIMESERIES_CONTEXT_LOS_ANGELES = ImmutableMap.of( - PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", - PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES, - "skipEmptyBuckets", true, - QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE - ); + public static final Map TIMESERIES_CONTEXT_LOS_ANGELES = Maps.newHashMap(); + { + TIMESERIES_CONTEXT_LOS_ANGELES.put(PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z"); + TIMESERIES_CONTEXT_LOS_ANGELES.put(PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES); + TIMESERIES_CONTEXT_LOS_ANGELES.put("skipEmptyBuckets", true); + TIMESERIES_CONTEXT_LOS_ANGELES.put(QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS); + TIMESERIES_CONTEXT_LOS_ANGELES.put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE); + TIMESERIES_CONTEXT_LOS_ANGELES.put( + PlannerContext.CTX_AUTHENTICATION_RESULT, + AllowAllAuthenticator.ALLOW_ALL_RESULT + ); + } + private static final PagingSpec FIRST_PAGING_SPEC = new PagingSpec(null, 1000, true); @Rule @@ -5458,6 +5476,9 @@ private List getResults( final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig, viewManager); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + final PlannerFactory plannerFactory = new PlannerFactory( druidSchema, CalciteTests.createMockQueryLifecycleFactory(walker), @@ -5465,7 +5486,14 @@ private List getResults( macroTable, plannerConfig, new AuthConfig(), - null + new AuthenticatorMapper(defaultMap, "allowAll"), + new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + } ); viewManager.createView( diff --git a/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java index a0ec577305a8..9b1d95d018f6 100644 --- a/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -29,6 +30,9 @@ import io.druid.query.extraction.RegexDimExtractionFn; import io.druid.query.extraction.TimeFormatExtractionFn; import io.druid.segment.column.ValueType; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.planner.PlannerContext; @@ -58,10 +62,16 @@ public class ExpressionsTest { private static final DateTimeZone LOS_ANGELES = DateTimeZone.forID("America/Los_Angeles"); + private static final Map defaultMap = Maps.newHashMap(); + { + defaultMap.put("allowAll", new AllowAllAuthenticator()); + } + private final PlannerContext plannerContext = PlannerContext.create( CalciteTests.createOperatorTable(), CalciteTests.createExprMacroTable(), new PlannerConfig(), + new AuthorizerMapper(null), ImmutableMap.of() ); private final RowSignature rowSignature = RowSignature diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java index ade19aa5720b..ea6f2890763b 100644 --- a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java @@ -23,13 +23,20 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.math.expr.ExprMacroTable; import io.druid.query.QueryInterruptedException; import io.druid.query.ResourceLimitExceededException; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -42,6 +49,7 @@ import io.druid.sql.http.SqlQuery; import io.druid.sql.http.SqlResource; import org.apache.calcite.tools.ValidationException; +import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -49,6 +57,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import java.io.ByteArrayOutputStream; @@ -69,15 +78,42 @@ public class SqlResourceTest private SqlResource resource; + private HttpServletRequest req; + @Before public void setUp() throws Exception { Calcites.setSystemProperties(); walker = CalciteTests.createMockWalker(temporaryFolder.newFolder()); + final PlannerConfig plannerConfig = new PlannerConfig(); final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); + req = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) + .anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) + .anyTimes(); + EasyMock.replay(req); + + + AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; + + Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + AuthenticatorMapper authenticatorMapper = new AuthenticatorMapper(defaultMap, "allowAll"); + resource = new SqlResource( JSON_MAPPER, new PlannerFactory( @@ -87,7 +123,8 @@ public void setUp() throws Exception macroTable, plannerConfig, new AuthConfig(), - null + authenticatorMapper, + authorizerMapper ) ); } @@ -240,7 +277,7 @@ public void testResourceLimitExceeded() throws Exception // Returns either an error or a result. private Pair>> doPost(final SqlQuery query) throws Exception { - final Response response = resource.doPost(query, null); + final Response response = resource.doPost(query, req); if (response.getStatus() == 200) { final StreamingOutput output = (StreamingOutput) response.getEntity(); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java index 119342966bdf..b03b742be75d 100644 --- a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; import io.druid.data.input.InputRow; import io.druid.java.util.common.Intervals; import io.druid.query.aggregation.CountAggregatorFactory; @@ -32,6 +33,9 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; @@ -139,11 +143,15 @@ public void setUp() throws Exception index2 ); + Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + schema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker), new TestServerInventoryView(walker.getSegments()), PLANNER_CONFIG_DEFAULT, - new NoopViewManager() + new NoopViewManager(), + new AuthenticatorMapper(defaultMap, "allowAll") ); schema.start(); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index 4ab0cb10d03a..22db930fe785 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -25,6 +25,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; @@ -87,7 +88,12 @@ import io.druid.server.QueryLifecycleFactory; import io.druid.server.initialization.ServerConfig; import io.druid.server.log.NoopRequestLogger; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.SqlOperatorConversion; @@ -105,6 +111,7 @@ import java.io.File; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -299,6 +306,8 @@ public static QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate() public static QueryLifecycleFactory createMockQueryLifecycleFactory(final QuerySegmentWalker walker) { + Map testAuthorizerMap = new HashMap<>(); + testAuthorizerMap.put("allowAll", new AllowAllAuthorizer()); return new QueryLifecycleFactory( new QueryToolChestWarehouse() { @@ -314,7 +323,7 @@ public > QueryToolChest getToolChest new NoopRequestLogger(), new ServerConfig(), new AuthConfig(), - new AuthorizerMapper(null) + new AuthorizerMapper(testAuthorizerMap) ); } @@ -398,11 +407,15 @@ public static DruidSchema createMockSchema( final ViewManager viewManager ) { + Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + final DruidSchema schema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker), new TestServerInventoryView(walker.getSegments()), plannerConfig, - viewManager + viewManager, + new AuthenticatorMapper(defaultMap, "allowAll") ); schema.start(); From 56c40dd87312312df7f4a68acba01c14ec3d09eb Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 11:36:15 -0700 Subject: [PATCH 18/34] Small cleanup --- docs/content/configuration/auth.md | 1 - .../security/kerberos/DruidKerberosModule.java | 16 ---------------- .../security/kerberos/KerberosAuthenticator.java | 1 - 3 files changed, 18 deletions(-) diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index 31c7e2278b5a..1e31f414171c 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -6,7 +6,6 @@ layout: doc_page |Property|Type|Description|Default|Required| |--------|-----------|--------|--------|--------| -|`druid.auth.enabled`|boolean|Determines if authentication and authorization checks will be performed on requests.|false|no| |`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|null|yes, if auth enabled| |`druid.auth.escalatedAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes, if auth enabled| |`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |null|yes, if auth enabled| diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java index e6c85fcc4f3e..e4058bc92d52 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosModule.java @@ -45,21 +45,5 @@ public List getJacksonModules() @Override public void configure(Binder binder) { - //JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", AuthenticationKerberosConfig.class); - //JsonConfigProvider.bind(binder, "druid.hadoop.security.spnego", SpnegoFilterConfig.class); } - - /* - @Provides - @ManageLifecycle - @Named("kerberos") - public Authenticator getAuthenticator( - AuthenticationKerberosConfig authenticationKerberosConfig, - SpnegoFilterConfig spnegoFilterConfig, - @Self DruidNode druidNode - ) - { - return new KerberosAuthenticator(authenticationKerberosConfig, spnegoFilterConfig, druidNode); - } - */ } diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index 71b1ae823e17..63b70d2cec5d 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -388,7 +388,6 @@ public Map getInitParameters() SecurityUtil.getServerPrincipal(serverPrincipal, node.getHost()) ); params.put("kerberos.keytab", serverKeytab); - //params.put(AuthenticationFilter.AUTH_TYPE, "kerberos"); params.put(AuthenticationFilter.AUTH_TYPE, DruidKerberosAuthenticationHandler.class.getName()); params.put("kerberos.name.rules", authToLocal); if (cookieSignatureSecret != null) { From 7ea9a1285d2c157db00438ae4c456c03d0e20570 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 13:00:23 -0700 Subject: [PATCH 19/34] Add Jetty HttpClient wrapper to Authenticator --- docs/content/configuration/auth.md | 6 +- .../kerberos/KerberosAuthenticator.java | 70 ++++++++++ .../KerberosJettyHttpClientProvider.java | 120 ------------------ .../server/AsyncQueryForwardingServlet.java | 11 +- .../security/AllowAllAuthenticator.java | 6 + .../druid/server/security/Authenticator.java | 16 ++- 6 files changed, 104 insertions(+), 125 deletions(-) delete mode 100644 extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index 1e31f414171c..0a75a8e85aa9 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -72,16 +72,20 @@ Allowing the user to redefine what identity string represents the internal syste ### Authenticator Internal System User Handling -Authenticators must implement two methods related to the internal system user: +Authenticators must implement three methods related to the internal system user: ```java public HttpClient createEscalatedClient(HttpClient baseClient); + public org.eclipse.jetty.client.HttpClient createEscalatedJettyClient(org.eclipse.jetty.client.HttpClient baseClient); + public AuthenticationResult createEscalatedAuthenticationResult(); ``` `createEscalatedClient` returns an wrapped HttpClient that attaches the credentials of the "internal system user" to requests. +`createEscalatedJettyClient` is similar to `createEscalatedClient`, except that it operates on a Jetty HttpClient. + `createEscalatedAuthenticationResult` returns an AuthenticationResult containing the identity of the "internal system user". As with Authenticators, we recommend that the "internal system user" be represented by default with the identity string "__DRUID_INTERNAL_SYSTEM". This is a guideline and not enforced. diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index 63b70d2cec5d..74a9549cfae0 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -33,6 +33,7 @@ import io.druid.server.security.Authenticator; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.client.AuthenticatedURL; import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.apache.hadoop.security.authentication.server.AuthenticationFilter; @@ -41,6 +42,11 @@ import org.apache.hadoop.security.authentication.util.Signer; import org.apache.hadoop.security.authentication.util.SignerException; import org.apache.hadoop.security.authentication.util.SignerSecretProvider; +import org.eclipse.jetty.client.api.Authentication; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.client.api.Request; +import org.eclipse.jetty.util.Attributes; +import org.jboss.netty.handler.codec.http.HttpHeaders; import sun.security.krb5.EncryptedData; import sun.security.krb5.EncryptionKey; import sun.security.krb5.internal.APReq; @@ -72,7 +78,9 @@ import javax.servlet.http.HttpServletResponse; import java.io.File; import java.io.IOException; +import java.net.URI; import java.security.Principal; +import java.security.PrivilegedExceptionAction; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -430,6 +438,68 @@ public HttpClient createEscalatedClient(HttpClient baseClient) return new KerberosHttpClient(baseClient, internalClientPrincipal, internalClientKeytab); } + @Override + public org.eclipse.jetty.client.HttpClient createEscalatedJettyClient(org.eclipse.jetty.client.HttpClient baseClient) + { + baseClient.getAuthenticationStore().addAuthentication(new Authentication() + { + @Override + public boolean matches(String type, URI uri, String realm) + { + return true; + } + + @Override + public Result authenticate( + final Request request, ContentResponse response, Authentication.HeaderInfo headerInfo, Attributes context + ) + { + return new Result() + { + @Override + public URI getURI() + { + return request.getURI(); + } + + @Override + public void apply(Request request) + { + try { + // No need to set cookies as they are handled by Jetty Http Client itself. + URI uri = request.getURI(); + if (DruidKerberosUtil.needToSendCredentials(baseClient.getCookieStore(), uri)) { + log.debug( + "No Auth Cookie found for URI[%s]. Existing Cookies[%s] Authenticating... ", + uri, + baseClient.getCookieStore().getCookies() + ); + final String host = request.getHost(); + DruidKerberosUtil.authenticateIfRequired(internalClientPrincipal, internalClientKeytab); + UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + String challenge = currentUser.doAs(new PrivilegedExceptionAction() + { + @Override + public String run() throws Exception + { + return DruidKerberosUtil.kerberosChallenge(host); + } + }); + request.getHeaders().add(HttpHeaders.Names.AUTHORIZATION, "Negotiate " + challenge); + } else { + log.debug("Found Auth Cookie found for URI[%s].", uri); + } + } + catch (Throwable e) { + Throwables.propagate(e); + } + } + }; + } + }); + return baseClient; + } + @Override public AuthenticationResult createEscalatedAuthenticationResult() { diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java deleted file mode 100644 index 1a60cfc623fb..000000000000 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosJettyHttpClientProvider.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.security.kerberos; - -import io.druid.guice.http.AbstractHttpClientProvider; -import org.eclipse.jetty.client.HttpClient; - -public class KerberosJettyHttpClientProvider extends AbstractHttpClientProvider -{ - @Override - public HttpClient get() - { - return null; - } - /* - private static final Logger log = new Logger(KerberosJettyHttpClientProvider.class); - - private final Provider delegateProvider; - private AuthenticationKerberosConfig config; - - - public KerberosJettyHttpClientProvider( - Provider delegateProvider - ) - { - this.delegateProvider = delegateProvider; - } - - @Inject - @Override - public void configure(Injector injector) - { - if (delegateProvider instanceof AbstractHttpClientProvider) { - ((AbstractHttpClientProvider) delegateProvider).configure(injector); - } - config = injector.getInstance(AuthenticationKerberosConfig.class); - } - - - @Override - public HttpClient get() - { - final HttpClient httpClient = delegateProvider.get(); - httpClient.getAuthenticationStore().addAuthentication(new Authentication() - { - @Override - public boolean matches(String type, URI uri, String realm) - { - return true; - } - - @Override - public Result authenticate( - final Request request, ContentResponse response, Authentication.HeaderInfo headerInfo, Attributes context - ) - { - return new Result() - { - @Override - public URI getURI() - { - return request.getURI(); - } - - @Override - public void apply(Request request) - { - try { - // No need to set cookies as they are handled by Jetty Http Client itself. - URI uri = request.getURI(); - if (DruidKerberosUtil.needToSendCredentials(httpClient.getCookieStore(), uri)) { - log.debug( - "No Auth Cookie found for URI[%s]. Existing Cookies[%s] Authenticating... ", - uri, - httpClient.getCookieStore().getCookies() - ); - final String host = request.getHost(); - DruidKerberosUtil.authenticateIfRequired(config); - UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); - String challenge = currentUser.doAs(new PrivilegedExceptionAction() - { - @Override - public String run() throws Exception - { - return DruidKerberosUtil.kerberosChallenge(host); - } - }); - request.getHeaders().add(HttpHeaders.Names.AUTHORIZATION, "Negotiate " + challenge); - } else { - log.debug("Found Auth Cookie found for URI[%s].", uri); - } - } - catch (Throwable e) { - Throwables.propagate(e); - } - } - }; - } - }); - return httpClient; - } - */ -} diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 849dfd82e686..8eec1b4ee924 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -42,6 +42,8 @@ import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.router.QueryHostFinder; import io.druid.server.router.Router; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.api.Request; import org.eclipse.jetty.client.api.Response; @@ -108,6 +110,7 @@ private static void handleException(HttpServletResponse response, ObjectMapper o private final ServiceEmitter emitter; private final RequestLogger requestLogger; private final GenericQueryMetricsFactory queryMetricsFactory; + private final Authenticator escalatingAuthenticator; private HttpClient broadcastClient; @@ -121,7 +124,8 @@ public AsyncQueryForwardingServlet( @Router DruidHttpClientConfig httpClientConfig, ServiceEmitter emitter, RequestLogger requestLogger, - GenericQueryMetricsFactory queryMetricsFactory + GenericQueryMetricsFactory queryMetricsFactory, + AuthenticatorMapper authenticatorMapper ) { this.warehouse = warehouse; @@ -133,6 +137,7 @@ public AsyncQueryForwardingServlet( this.emitter = emitter; this.requestLogger = requestLogger; this.queryMetricsFactory = queryMetricsFactory; + this.escalatingAuthenticator = authenticatorMapper.getEscalatingAuthenticator(); } @Override @@ -142,7 +147,7 @@ public void init() throws ServletException // Note that httpClientProvider is setup to return same HttpClient instance on each get() so // it is same http client as that is used by parent ProxyServlet. - broadcastClient = httpClientProvider.get(); + broadcastClient = newHttpClient(); try { broadcastClient.start(); } @@ -317,7 +322,7 @@ protected static URI makeURI(String scheme, String host, String requestURI, Stri @Override protected HttpClient newHttpClient() { - return httpClientProvider.get(); + return escalatingAuthenticator.createEscalatedJettyClient(httpClientProvider.get()); } @Override diff --git a/server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java b/server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java index 99fec24ef653..b6a9ae8673d9 100644 --- a/server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java +++ b/server/src/main/java/io/druid/server/security/AllowAllAuthenticator.java @@ -109,6 +109,12 @@ public HttpClient createEscalatedClient(HttpClient baseClient) return baseClient; } + @Override + public org.eclipse.jetty.client.HttpClient createEscalatedJettyClient(org.eclipse.jetty.client.HttpClient baseClient) + { + return baseClient; + } + @Override public AuthenticationResult createEscalatedAuthenticationResult() { diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index 49426584d682..aaf63b3d6bba 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -38,6 +38,8 @@ * authentication mechanism, getAuthChallengeHeader(). * - A method for creating a wrapped HTTP client that can authenticate using the Authenticator's authentication scheme, * used for internal Druid node communications (e.g., broker -> historical messages), createEscalatedClient(). + * - A method for creating a wrapped Jetty HTTP client that can authenticate using the Authenticator's authentication scheme, + * used by the Druid router, createEscalatedJettyClient(). * - A method for authenticating credentials contained in a JDBC connection context, used for authenticating Druid SQL * requests received via JDBC, authenticateJDBCContext(). */ @@ -96,10 +98,22 @@ public interface Authenticator extends ServletFilterHolder * * @param baseClient Base HTTP client for internal Druid communications * - * @return HttpClient that sends requests with the credentials of the internal system user + * @return metamx HttpClient that sends requests with the credentials of the internal system user */ public HttpClient createEscalatedClient(HttpClient baseClient); + /** + * Return a client that sends requests with the format/information necessary to authenticate successfully + * against this Authenticator's authentication scheme using the identity of the internal system user. + *

+ * This HTTP client is used by the Druid Router node. + * + * @param baseClient Base Jetty HttpClient + * + * @return Jetty HttpClient that sends requests with the credentials of the internal system user + */ + public org.eclipse.jetty.client.HttpClient createEscalatedJettyClient(org.eclipse.jetty.client.HttpClient baseClient); + /** * @return an AuthenticationResult representing the identity of the internal system user. */ From 042a4bbf2aca6ca073da50dafd145f49ee740746 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 13:07:08 -0700 Subject: [PATCH 20/34] Remove Authorizer start/stop --- .../server/security/AllowAllAuthorizer.java | 1 - .../io/druid/server/security/Authorizer.java | 16 ---------------- .../druid/server/security/AuthorizerMapper.java | 6 ------ 3 files changed, 23 deletions(-) diff --git a/server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java b/server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java index 1b14f950d886..12bd6e6597cf 100644 --- a/server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java +++ b/server/src/main/java/io/druid/server/security/AllowAllAuthorizer.java @@ -26,5 +26,4 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso { return Access.OK; } - } diff --git a/server/src/main/java/io/druid/server/security/Authorizer.java b/server/src/main/java/io/druid/server/security/Authorizer.java index 12f64ebd8e3d..19bd9d41f8a2 100644 --- a/server/src/main/java/io/druid/server/security/Authorizer.java +++ b/server/src/main/java/io/druid/server/security/Authorizer.java @@ -50,20 +50,4 @@ public interface Authorizer * @return An Access object representing the result of the authorization check. */ Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action); - - /** - * Authorizers are registered with an AuthorizerMapper. The AuthorizerMapper is lifecycle managed and will - * call start() on each of its registered Authorizers in the AuthorizerMapper's start() method. - */ - default void start() - { - } - - /** - * Authorizers are registered with an AuthorizerMapper. The AuthorizerMapper is lifecycle managed and will - * call stop() on each of its registered Authorizers in the AuthorizerMapper's stop() method. - */ - default void stop() - { - } } diff --git a/server/src/main/java/io/druid/server/security/AuthorizerMapper.java b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java index ea0e6ba467ee..26487bf2697f 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizerMapper.java +++ b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java @@ -44,16 +44,10 @@ public Authorizer getAuthorizer(String name) @LifecycleStart public void start() { - for (Authorizer authorizer : authorizerMap.values()) { - authorizer.start(); - } } @LifecycleStart public void stop() { - for (Authorizer authorizer : authorizerMap.values()) { - authorizer.stop(); - } } } From 5c37dc708065867e9afee33867691106681185e4 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 13:30:18 -0700 Subject: [PATCH 21/34] Restore immutable context map in DruidConnection, UT fix --- .../AsyncQueryForwardingServletTest.java | 10 +++++++- .../io/druid/sql/avatica/DruidConnection.java | 6 ++--- .../java/io/druid/sql/avatica/DruidMeta.java | 24 ++++++------------- 3 files changed, 19 insertions(+), 21 deletions(-) diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index dbed7467dfc8..a0130c238710 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import com.google.common.net.HostAndPort; import com.google.inject.Binder; import com.google.inject.Inject; @@ -51,6 +52,9 @@ import io.druid.server.log.RequestLogger; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.router.QueryHostFinder; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.Authorizer; import io.druid.server.security.AllowAllAuthorizer; @@ -75,6 +79,7 @@ import java.net.URI; import java.net.URL; import java.util.Collection; +import java.util.Map; import java.util.concurrent.CountDownLatch; public class AsyncQueryForwardingServletTest extends BaseJettyTest @@ -233,6 +238,8 @@ public Collection getAllServers() } }; + Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); ObjectMapper jsonMapper = injector.getInstance(ObjectMapper.class); ServletHolder holder = new ServletHolder( new AsyncQueryForwardingServlet( @@ -251,7 +258,8 @@ public void log(RequestLogLine requestLogLine) throws IOException // noop } }, - new DefaultGenericQueryMetricsFactory(jsonMapper) + new DefaultGenericQueryMetricsFactory(jsonMapper), + new AuthenticatorMapper(defaultMap, "allowAll") ) { @Override diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java index 0af42ba916d1..2eb29ac7ccf6 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java @@ -21,7 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Maps; +import com.google.common.collect.ImmutableMap; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; @@ -41,7 +41,7 @@ public class DruidConnection private final String connectionId; private final int maxStatements; - private final Map context; + private final ImmutableMap context; private final AtomicInteger statementCounter = new AtomicInteger(); private final AtomicReference> timeoutFuture = new AtomicReference<>(); @@ -55,7 +55,7 @@ public DruidConnection(final String connectionId, final int maxStatements, final { this.connectionId = Preconditions.checkNotNull(connectionId); this.maxStatements = maxStatements; - this.context = Maps.newHashMap(context); + this.context = ImmutableMap.copyOf(context); this.statements = new HashMap<>(); } diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java index 7f742291b581..2f1d1116ffc4 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java @@ -145,11 +145,10 @@ public StatementHandle prepare( final StatementHandle statement = createStatement(ch); final DruidStatement druidStatement = getDruidStatement(statement); final DruidConnection druidConnection = getDruidConnection(statement.connectionId); - AuthenticationResult authenticationResult = null; - if (!authenticateConnection(druidConnection)) { + AuthenticationResult authenticationResult = authenticateConnection(druidConnection); + if (authenticationResult == null) { throw new SecurityException("Authentication failed."); } - authenticationResult = getConnectionAuthenticationResult(druidConnection); statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, authenticationResult).getSignature(); return statement; } @@ -179,12 +178,10 @@ public ExecuteResult prepareAndExecute( // Ignore "callback", this class is designed for use with LocalService which doesn't use it. final DruidStatement druidStatement = getDruidStatement(statement); final DruidConnection druidConnection = getDruidConnection(statement.connectionId); - AuthenticationResult authenticationResult = null; - if (!authenticateConnection(druidConnection)) { + AuthenticationResult authenticationResult = authenticateConnection(druidConnection); + if (authenticationResult == null) { throw new SecurityException("Authentication failed."); } - authenticationResult = getConnectionAuthenticationResult(druidConnection); - final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, authenticationResult).getSignature(); final Frame firstFrame = druidStatement.execute() .nextFrame( @@ -512,23 +509,16 @@ void closeAllConnections() } } - private boolean authenticateConnection(final DruidConnection connection) + private AuthenticationResult authenticateConnection(final DruidConnection connection) { Map context = connection.context(); for (Authenticator authenticator : authenticators) { AuthenticationResult authenticationResult = authenticator.authenticateJDBCContext(context); if (authenticationResult != null) { - context.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); - return true; + return authenticationResult; } } - return false; - } - - private AuthenticationResult getConnectionAuthenticationResult(final DruidConnection connection) - { - Map context = connection.context(); - return (AuthenticationResult) context.get(AuthConfig.DRUID_AUTHENTICATION_RESULT); + return null; } private DruidConnection openDruidConnection(final String connectionId, final Map context) From 880df860b7ca65ce4b4f835287269e219891f095 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 13:56:45 -0700 Subject: [PATCH 22/34] Fix/update docs --- docs/content/configuration/auth.md | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index 0a75a8e85aa9..db4c86991418 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -6,9 +6,9 @@ layout: doc_page |Property|Type|Description|Default|Required| |--------|-----------|--------|--------|--------| -|`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|null|yes, if auth enabled| -|`druid.auth.escalatedAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|null|yes, if auth enabled| -|`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |null|yes, if auth enabled| +|`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|[]|no| +|`druid.auth.escalatedAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|"allowAll"|no| +|`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |[]|no| ## Enabling Authentication/Authorization @@ -25,6 +25,12 @@ druid.auth.authenticationChain=["kerberos", "basic"] A request will pass through all Authenticators in the chain, unless one of the Authenticators sends an HTTP error response. If no Authenticator in the chain successfully authenticated a request, an HTTP error response will be sent. +Druid includes a built-in Authenticator, used for the default unsecured configuration. + +### AllowAll Authenticator + +This built-in Authenticator authenticates all requests, and always directs them to an Authorizer named "allowAll". It is not intended to be used for anything other than the default unsecured configuration. + ## Internal Authenticator The `druid.auth.escalatedAuthenticator` property determines what authentication scheme should be used for internal Druid cluster communications (such as when a broker node communicates with historical nodes for query processing). @@ -44,11 +50,21 @@ druid.auth.authorizers=["basic"] Only a single Authorizer will authorize any given request. -### Default Authorizer -The default Authorizer with type name "default" rejects all requests. +Druid includes two built in authorizers: + +### DenyAll Authorizer +The Authorizer with type name "denyAll" rejects all requests. + +### AllowAll Authorizer +The Authorizer with type name "allowAll" accepts all requests. + +## Default Unsecured Configuration + +When `druid.auth.authenticationChain` is left empty or unspecified, Druid will create an authentication chain with a single AllowAll Authenticator named "allowAll". + +When `druid.auth.authorizers` is left empty or unspecified, Druid will create a single AllowAll Authorizer named "allowAll". -### No-op Authorizer -The no-op Authorizer with type name "noop" accepts all requests. +The default value of `druid.auth.escalatedAuthenticator` is "allowAll" to match the default unsecured Authenticator/Authorizer configurations. ## Authenticator to Authorizer Routing From a3343ee80069908f4d95e3b1ff1facb5c8af2911 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 31 Aug 2017 16:38:53 -0700 Subject: [PATCH 23/34] Add authorization checks to EventReceiverFirehose --- .../EventReceiverFirehoseFactory.java | 39 ++++++++++++++- .../firehose/EventReceiverFirehoseTest.java | 48 +++++++++++++++++-- 2 files changed, 81 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index 8b280dd4ec41..80cd4165f650 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -44,6 +44,13 @@ import io.druid.java.util.common.DateTimes; import io.druid.server.metrics.EventReceiverFirehoseMetric; import io.druid.server.metrics.EventReceiverFirehoseRegister; +import io.druid.server.security.Access; +import io.druid.server.security.Action; +import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.Resource; +import io.druid.server.security.ResourceAction; +import io.druid.server.security.ResourceType; import org.joda.time.DateTime; import javax.servlet.http.HttpServletRequest; @@ -83,6 +90,7 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory rows) throws InterruptedException @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public Response shutdown( - @QueryParam("shutoffTime") final String shutoffTime + @QueryParam("shutoffTime") final String shutoffTime, + @Context final HttpServletRequest req ) { + Access accessResult = AuthorizationUtils.authorizeResourceAction( + req, + new ResourceAction( + new Resource("STATE", ResourceType.STATE), + Action.WRITE + ), + authorizerMapper + ); + if (!accessResult.isAllowed()) { + return Response.status(403).build(); + } + try { DateTime shutoffAt = shutoffTime == null ? DateTimes.nowUtc() : DateTimes.of(shutoffTime); log.info("Setting Firehose shutoffTime to %s", shutoffTime); diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index 94c87c953c37..0152a183d203 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -31,6 +31,11 @@ import io.druid.java.util.common.ISE; import io.druid.server.metrics.EventReceiverFirehoseMetric; import io.druid.server.metrics.EventReceiverFirehoseRegister; +import io.druid.server.security.AllowAllAuthenticator; +import io.druid.server.security.AllowAllAuthorizer; +import io.druid.server.security.AuthConfig; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import org.apache.commons.io.IOUtils; import org.easymock.EasyMock; import org.junit.Assert; @@ -63,6 +68,14 @@ public class EventReceiverFirehoseTest private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); private HttpServletRequest req; + private AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) + { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; @Before public void setUp() throws Exception @@ -74,7 +87,8 @@ public void setUp() throws Exception null, new DefaultObjectMapper(), new DefaultObjectMapper(), - register + register, + authorizerMapper ); firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( new MapInputRowParser( @@ -95,6 +109,11 @@ public void setUp() throws Exception @Test public void testSingleThread() throws IOException { + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) + .anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); EasyMock.expect(req.getContentType()).andReturn("application/json").times(NUM_EVENTS); EasyMock.replay(req); @@ -137,6 +156,12 @@ public void testSingleThread() throws IOException @Test public void testMultipleThreads() throws InterruptedException, IOException, TimeoutException, ExecutionException { + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) + .anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(req.getContentType()).andReturn("application/json").times(2 * NUM_EVENTS); EasyMock.replay(req); @@ -205,7 +230,8 @@ public void testDuplicateRegistering() throws IOException null, new DefaultObjectMapper(), new DefaultObjectMapper(), - register + register, + authorizerMapper ); EventReceiverFirehoseFactory.EventReceiverFirehose firehose2 = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory2 @@ -228,7 +254,14 @@ public void testDuplicateRegistering() throws IOException @Test(timeout = 40_000L) public void testShutdownWithPrevTime() throws Exception { - firehose.shutdown(DateTimes.nowUtc().minusMinutes(2).toString()); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) + .anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + EasyMock.replay(req); + + firehose.shutdown(DateTimes.nowUtc().minusMinutes(2).toString(), req); while (!firehose.isClosed()) { Thread.sleep(50); } @@ -237,7 +270,14 @@ public void testShutdownWithPrevTime() throws Exception @Test(timeout = 40_000L) public void testShutdown() throws Exception { - firehose.shutdown(DateTimes.nowUtc().plusMillis(100).toString()); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) + .anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + EasyMock.replay(req); + + firehose.shutdown(DateTimes.nowUtc().plusMillis(100).toString(), req); while (!firehose.isClosed()) { Thread.sleep(50); } From 9734c9d2d2cf854042b276227464810300868a9e Mon Sep 17 00:00:00 2001 From: jon-wei Date: Tue, 5 Sep 2017 09:19:41 -0700 Subject: [PATCH 24/34] Fix router authorization check failure, restore PreResponseAuthorizationFilter changes --- .../server/AsyncQueryForwardingServlet.java | 7 +++++++ .../PreResponseAuthorizationCheckFilter.java | 20 +++++++++---------- 2 files changed, 16 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 8eec1b4ee924..39e16044ffb9 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -42,6 +42,7 @@ import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.router.QueryHostFinder; import io.druid.server.router.Router; +import io.druid.server.security.AuthConfig; import io.druid.server.security.Authenticator; import io.druid.server.security.AuthenticatorMapper; import org.eclipse.jetty.client.HttpClient; @@ -271,6 +272,12 @@ protected void sendProxyRequest( } } + // Since we can't see the request object on the remote side, we can't check whether the remote side actually + // performed an authorization check here, so always set this to true for the proxy servlet. + // If the remote node failed to perform an authorization check, PreResponseAuthorizationCheckFilter + // will log that on the remote node. + clientRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + super.sendProxyRequest( clientRequest, proxyResponse, diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index 8036b6848420..6a5273f69139 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -77,16 +77,6 @@ public void doFilter( ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain ) throws IOException, ServletException { - QueryInterruptedException unauthorizedError = new QueryInterruptedException( - QueryInterruptedException.UNAUTHORIZED, - null, - null, - DruidNode.getDefaultHost() - ); - unauthorizedError.setStackTrace(new StackTraceElement[0]); - OutputStream out = servletResponse.getOutputStream(); - - Boolean authInfoChecked = null; final HttpServletResponse response = (HttpServletResponse) servletResponse; // Since this is the last filter in the chain, some previous authentication filter @@ -103,6 +93,14 @@ public void doFilter( for (String authScheme : supportedAuthSchemes) { response.addHeader("WWW-Authenticate", authScheme); } + QueryInterruptedException unauthorizedError = new QueryInterruptedException( + QueryInterruptedException.UNAUTHORIZED, + null, + null, + DruidNode.getDefaultHost() + ); + unauthorizedError.setStackTrace(new StackTraceElement[0]); + OutputStream out = servletResponse.getOutputStream(); sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); out.close(); return; @@ -110,7 +108,7 @@ public void doFilter( filterChain.doFilter(servletRequest, servletResponse); - authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED); + Boolean authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED); if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { String errorMsg = StringUtils.format( "Request did not have an authorization check performed: %s", From 1686d8fd43b305be70ed2214b2ee2c89f35583df Mon Sep 17 00:00:00 2001 From: jon-wei Date: Tue, 5 Sep 2017 09:38:11 -0700 Subject: [PATCH 25/34] Compile fixes --- .../http/security/PreResponseAuthorizationCheckFilterTest.java | 2 -- .../main/java/io/druid/sql/calcite/rel/DruidOuterQueryRel.java | 2 +- sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java | 3 ++- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index 31e09d7fa9a3..dec4220cff16 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -56,7 +56,6 @@ public void testValidRequest() throws Exception FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(authenticationResult).once(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(true).once(); EasyMock.replay(req, resp, filterChain, outputStream); @@ -110,7 +109,6 @@ public void testMissingAuthorizationCheck() throws Exception FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(authenticationResult).once(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).once(); EasyMock.expect(resp.getStatus()).andReturn(200).once(); diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidOuterQueryRel.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidOuterQueryRel.java index eec2e5909850..e2456be5f105 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidOuterQueryRel.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidOuterQueryRel.java @@ -184,7 +184,7 @@ public RelNode copy(final RelTraitSet traitSet, final List inputs) @Override public List getDatasourceNames() { - return sourceRel.getDatasourceNames(); + return ((DruidRel) sourceRel).getDatasourceNames(); } @Override diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java index 2ab4862e847b..0ef006036590 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidSemiJoin.java @@ -197,9 +197,10 @@ public DruidSemiJoin asDruidConvention() @Override public List getDatasourceNames() { + final DruidRel druidRight = (DruidRel) this.right; Set datasourceNames = Sets.newLinkedHashSet(); datasourceNames.addAll(left.getDatasourceNames()); - datasourceNames.addAll(right.getDatasourceNames()); + datasourceNames.addAll(druidRight.getDatasourceNames()); return Lists.newArrayList(datasourceNames); } From b9c564d39b72d22d8c9f03bdc2bd1f82117cba64 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Tue, 5 Sep 2017 21:21:30 -0700 Subject: [PATCH 26/34] Test fixes --- .../sql/QuantileSqlAggregatorTest.java | 7 +-- .../io/druid/sql/avatica/DruidConnection.java | 43 ++++++++++++++++--- .../sql/calcite/planner/DruidPlanner.java | 19 ++++---- .../sql/calcite/planner/PlannerContext.java | 13 ++++++ .../sql/calcite/rel/DruidQueryBuilder.java | 11 ++--- .../io/druid/sql/calcite/rel/QueryMaker.java | 5 +-- .../druid/sql/calcite/CalciteQueryTest.java | 37 ++++++---------- 7 files changed, 82 insertions(+), 53 deletions(-) diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index febdabf836dc..f4da8a11243a 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -57,7 +57,6 @@ import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerConfig; -import io.druid.sql.calcite.planner.PlannerContext; import io.druid.sql.calcite.planner.PlannerFactory; import io.druid.sql.calcite.planner.PlannerResult; import io.druid.sql.calcite.schema.DruidSchema; @@ -243,8 +242,7 @@ public void testQuantileOnFloatAndLongs() throws Exception new QuantilePostAggregator("a8", "a8:agg", 0.50f) )) .context(ImmutableMap.of( - "skipEmptyBuckets", true, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + "skipEmptyBuckets", true )) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) @@ -306,8 +304,7 @@ public void testQuantileOnComplexColumn() throws Exception new QuantilePostAggregator("a6", "a4:agg", 0.999f) )) .context(ImmutableMap.of( - "skipEmptyBuckets", true, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + "skipEmptyBuckets", true )) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java index 2eb29ac7ccf6..8af20c2b8caa 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidConnection.java @@ -20,14 +20,20 @@ package io.druid.sql.avatica; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSortedMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -38,6 +44,9 @@ public class DruidConnection { private static final Logger log = new Logger(DruidConnection.class); + private static final Set SENSITIVE_CONTEXT_FIELDS = Sets.newHashSet( + "user", "password" + ); private final String connectionId; private final int maxStatements; @@ -74,13 +83,33 @@ public DruidStatement createStatement() throw new ISE("Too many open statements, limit is[%,d]", maxStatements); } - final DruidStatement statement = new DruidStatement(connectionId, statementId, context, () -> { - // onClose function for the statement - synchronized (statements) { - log.debug("Connection[%s] closed statement[%s].", connectionId, statementId); - statements.remove(statementId); - } - }); + // remove sensitive fields from the context, only the connection's context needs to have authentication + // credentials + Map sanitizedContext = Maps.newHashMap(); + sanitizedContext = Maps.filterEntries( + context, + new Predicate>() + { + @Override + public boolean apply(@Nullable Map.Entry input) + { + return !SENSITIVE_CONTEXT_FIELDS.contains(input.getKey()); + } + } + ); + + final DruidStatement statement = new DruidStatement( + connectionId, + statementId, + ImmutableSortedMap.copyOf(sanitizedContext), + () -> { + // onClose function for the statement + synchronized (statements) { + log.debug("Connection[%s] closed statement[%s].", connectionId, statementId); + statements.remove(statementId); + } + } + ); statements.put(statementId, statement); log.debug("Connection[%s] opened statement[%s].", connectionId, statementId); diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java index 6704205855fb..d452d8f8f9d0 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java @@ -162,11 +162,9 @@ private PlannerResult planWithDruidConvention( AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, authorizerMapper ); - plannerContext.getQueryContext() - .put( - PlannerContext.CTX_AUTHENTICATION_RESULT, - request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) - ); + plannerContext.setAuthenticationResult( + (AuthenticationResult) request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) + ); } else { authResult = AuthorizationUtils.authorizeAllResourceActions( datasourceNames, @@ -174,11 +172,9 @@ private PlannerResult planWithDruidConvention( authenticationResult, authorizerMapper ); - plannerContext.getQueryContext().put(PlannerContext.CTX_AUTHENTICATION_RESULT, authenticationResult); + plannerContext.setAuthenticationResult(authenticationResult); } - - if (!authResult.isAllowed()) { throw new SecurityException(authResult.toString()); } @@ -219,6 +215,7 @@ public Object[] apply(final Object[] input) private Access authorizeBindableRel( BindableRel rel, + final PlannerContext plannerContext, HttpServletRequest req, final AuthenticationResult authenticationResult ) @@ -244,6 +241,9 @@ public void visit(RelNode node, int ordinal, RelNode parent) } ); if (req != null) { + plannerContext.setAuthenticationResult( + (AuthenticationResult) req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) + ); return AuthorizationUtils.authorizeAllResourceActions( req, datasourceNames, @@ -251,6 +251,7 @@ public void visit(RelNode node, int ordinal, RelNode parent) authorizerMapper ); } else { + plannerContext.setAuthenticationResult(authenticationResult); return AuthorizationUtils.authorizeAllResourceActions( datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, @@ -291,7 +292,7 @@ private PlannerResult planWithBindableConvention( ); } - Access accessResult = authorizeBindableRel(bindableRel, request, authenticationResult); + Access accessResult = authorizeBindableRel(bindableRel, plannerContext, request, authenticationResult); if (!accessResult.isAllowed()) { throw new SecurityException(accessResult.toString()); } diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java index d1a5b36cfd99..a385a0be1ba9 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/PlannerContext.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import io.druid.math.expr.ExprMacroTable; +import io.druid.server.security.AuthenticationResult; import io.druid.server.security.AuthorizerMapper; import org.apache.calcite.DataContext; import org.apache.calcite.adapter.java.JavaTypeFactory; @@ -52,6 +53,8 @@ public class PlannerContext private final Map queryContext; private final AuthorizerMapper authorizerMapper; + private AuthenticationResult authenticationResult; + private PlannerContext( final DruidOperatorTable operatorTable, final ExprMacroTable macroTable, @@ -146,6 +149,16 @@ public long getQueryStartTimeMillis() return queryStartTimeMillis; } + public AuthenticationResult getAuthenticationResult() + { + return authenticationResult; + } + + public void setAuthenticationResult(AuthenticationResult authenticationResult) + { + this.authenticationResult = authenticationResult; + } + public DataContext createDataContext(final JavaTypeFactory typeFactory) { class DruidDataContext implements DataContext diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java index 212d0270ecc9..06a305a7357b 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSortedMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -445,7 +446,7 @@ public TimeseriesQuery toTimeseriesQuery( descending = false; } - final Map theContext = Maps.newHashMap(); + Map theContext = Maps.newHashMap(); theContext.put("skipEmptyBuckets", true); theContext.putAll(plannerContext.getQueryContext()); @@ -458,7 +459,7 @@ public TimeseriesQuery toTimeseriesQuery( queryGranularity, grouping.getAggregatorFactories(), grouping.getPostAggregators(), - theContext + ImmutableSortedMap.copyOf(theContext) ); } @@ -533,7 +534,7 @@ public TopNQuery toTopNQuery( Granularities.ALL, grouping.getAggregatorFactories(), grouping.getPostAggregators(), - plannerContext.getQueryContext() + ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) ); } @@ -568,7 +569,7 @@ public GroupByQuery toGroupByQuery( grouping.getPostAggregators(), having != null ? new DimFilterHavingSpec(having) : null, limitSpec, - plannerContext.getQueryContext() + ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) ); } @@ -649,7 +650,7 @@ public SelectQuery toSelectQuery( metrics.stream().sorted().distinct().collect(Collectors.toList()), getVirtualColumns(plannerContext.getExprMacroTable()), pagingSpec, - plannerContext.getQueryContext() + ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) ); } diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java index 17c5001b1acd..927826e42f16 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/QueryMaker.java @@ -221,10 +221,7 @@ public void remove() private Sequence runQuery(final Query query) { Hook.QUERY_PLAN.run(query); - - final AuthenticationResult authenticationResult = - (AuthenticationResult) plannerContext.getQueryContext().get(PlannerContext.CTX_AUTHENTICATION_RESULT); - + final AuthenticationResult authenticationResult = plannerContext.getAuthenticationResult(); return queryLifecycleFactory.factorize().runSimple(query, authenticationResult, null); } diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index 175a538961bb..361c97b49fc8 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -182,32 +182,28 @@ public int getMaxQueryCount() private static final Map QUERY_CONTEXT_DEFAULT = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final Map QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", "skipEmptyBuckets", false, QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final Map QUERY_CONTEXT_NO_TOPN = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, "false", QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); private static final Map QUERY_CONTEXT_LOS_ANGELES = ImmutableMap.of( PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES, QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); // Matches QUERY_CONTEXT_DEFAULT @@ -215,8 +211,7 @@ public int getMaxQueryCount() PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z", "skipEmptyBuckets", true, QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS, - QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE, - PlannerContext.CTX_AUTHENTICATION_RESULT, AllowAllAuthenticator.ALLOW_ALL_RESULT + QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); // Matches QUERY_CONTEXT_LOS_ANGELES @@ -227,10 +222,6 @@ public int getMaxQueryCount() TIMESERIES_CONTEXT_LOS_ANGELES.put("skipEmptyBuckets", true); TIMESERIES_CONTEXT_LOS_ANGELES.put(QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS); TIMESERIES_CONTEXT_LOS_ANGELES.put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE); - TIMESERIES_CONTEXT_LOS_ANGELES.put( - PlannerContext.CTX_AUTHENTICATION_RESULT, - AllowAllAuthenticator.ALLOW_ALL_RESULT - ); } private static final PagingSpec FIRST_PAGING_SPEC = new PagingSpec(null, 1000, true); @@ -437,7 +428,7 @@ public void testExplainSelectStar() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n" + "DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}])\n" } ) ); @@ -661,8 +652,8 @@ public void testExplainSelfJoinWithFallback() throws Exception new Object[]{ "BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n" + " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807}}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"select\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dummy\",\"outputName\":\"dummy\",\"outputType\":\"STRING\"}],\"metrics\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"virtualColumns\":[],\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":0,\"fromNext\":true},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"}}])\n" } ) ); @@ -3281,9 +3272,9 @@ public void testExplainDoubleNestedGroupBy() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null},{\"type\":\"count\",\"name\":\"a1\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" - + " DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"d1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d1\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" + "DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null},{\"type\":\"count\",\"name\":\"a1\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"d1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"a0\",\"expression\":null}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d1\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); @@ -3512,9 +3503,9 @@ public void testExplainExactCountDistinctOfSemiJoinResult() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" - + " DruidSemiJoin(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}], leftExpressions=[[DruidExpression{simpleExtraction=null, expression='substring(\"dim2\", 0, 1)'}]], rightKeys=[[0]])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" + "DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidSemiJoin(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}], leftExpressions=[[DruidExpression{simpleExtraction=null, expression='substring(\"dim2\", 0, 1)'}]], rightKeys=[[0]])\n" + + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); From 02e0a729607eb0fba4759648c5b9fd3339c84eaf Mon Sep 17 00:00:00 2001 From: jon-wei Date: Thu, 7 Sep 2017 12:28:59 -0700 Subject: [PATCH 27/34] Update Authenticator/Authorizer doc comments --- .../io/druid/server/security/AuthConfig.java | 2 +- .../druid/server/security/Authenticator.java | 19 +++++++++++-------- .../io/druid/server/security/Authorizer.java | 2 +- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index b464b8d4edab..79be88406b16 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -34,7 +34,7 @@ public class AuthConfig /** * HTTP attribute set when a static method in AuthorizationUtils performs an authorization check on the request. */ - public static final String DRUID_AUTHORIZATION_CHECKED = "Druid-Auth-Token-Checked"; + public static final String DRUID_AUTHORIZATION_CHECKED = "Druid-Authorization-Checked"; public AuthConfig() { diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index aaf63b3d6bba..1fe925e117b2 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -48,17 +48,20 @@ public interface Authenticator extends ServletFilterHolder /** * Create a Filter that performs authentication checks on incoming HTTP requests. *

- * If the authentication succeeds, the Filter should set the "Druid-Auth-Token" attribute in the request, - * containing a String that represents the authenticated identity of the requester. + * If the authentication succeeds, the Filter should set the "Druid-Authentication-Result" attribute in the request, + * containing an AuthenticationResult that represents the authenticated identity of the requester, along with + * the name of the Authorizer instance that should authorize the request. *

- * If the "Druid-Auth-Token" attribute is already set (i.e., request has been authenticated by an earlier Filter), - * this Filter should skip any authentication checks and proceed to the next Filter. + * If the "Druid-Authentication-Result" attribute is already set (i.e., request has been authenticated by an + * earlier Filter), this Filter should skip any authentication checks and proceed to the next Filter. *

- * If the authentication fails, the Filter should not send an error response. The error response will be sent - * after all Filters in the authentication filter chain have been checked. + * If a filter cannot recognize a request's format (e.g., the request does not have credentials compatible + * with a filter's authentication scheme), the filter should not send an error response, allowing other + * filters to handle the request. A challenge response will be sent if the filter chain is exhausted. *

- * If an anonymous request is received, the Filter should continue on to the next Filter, the challenge response - * will be sent after the filter chain is exhausted. + * If the authentication fails (i.e., a filter recognized the authentication scheme of a request, but the credentials + * failed to authenticate successfully) the Filter should send an error response, without needing to proceed to + * other filters in the chain.. * * @return Filter that authenticates HTTP requests */ diff --git a/server/src/main/java/io/druid/server/security/Authorizer.java b/server/src/main/java/io/druid/server/security/Authorizer.java index 19bd9d41f8a2..3580bdb51857 100644 --- a/server/src/main/java/io/druid/server/security/Authorizer.java +++ b/server/src/main/java/io/druid/server/security/Authorizer.java @@ -35,7 +35,7 @@ * attribute, previously set by an Authenticator. Each endpoint will pass this identity String to the * Authorizer's authorize() method along with any Resource/Action pairs created for the request being * handled. The endpoint can use these checks to filter out resources or deny the request as needed. - * After a request is authorized, a new attribute, "Druid-Auth-Token-Checked", should be set in the + * After a request is authorized, a new attribute, "Druid-Authorization-Checked", should be set in the * request header with the result of the authorization decision. */ public interface Authorizer From e9188a0809dc09abf7839f6a1e4d49325ccddcb5 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Mon, 11 Sep 2017 13:24:48 -0700 Subject: [PATCH 28/34] Merge fixes --- .../common/actions/RemoteTaskActionClientFactory.java | 7 ++----- .../io/druid/client/coordinator/CoordinatorClient.java | 1 - .../io/druid/server/router/CoordinatorRuleManager.java | 1 - .../java/io/druid/discovery/DruidLeaderClientTest.java | 1 - .../coordinator/DruidCoordinatorSegmentMergerTest.java | 2 -- .../druid/server/router/TieredBrokerHostSelectorTest.java | 2 -- 6 files changed, 2 insertions(+), 12 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java index 95ae370614d8..efb5f6189319 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/RemoteTaskActionClientFactory.java @@ -25,7 +25,6 @@ import io.druid.discovery.DruidLeaderClient; import io.druid.indexing.common.RetryPolicyFactory; import io.druid.indexing.common.task.Task; -import io.druid.server.security.AuthenticatorHttpClientWrapper; /** */ @@ -38,12 +37,10 @@ public class RemoteTaskActionClientFactory implements TaskActionClientFactory @Inject public RemoteTaskActionClientFactory( @IndexingService DruidLeaderClient leaderHttpClient, - RetryPolicyFactory retryPoslicyFactory, - ObjectMapper jsonMapper, - AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + RetryPolicyFactory retryPolicyFactory, + ObjectMapper jsonMapper ) { - this.druidLeaderClient = authenticatorHttpClientWrapper.getEscalatedClient(leaderHttpClient); this.druidLeaderClient = leaderHttpClient; this.retryPolicyFactory = retryPolicyFactory; this.jsonMapper = jsonMapper; diff --git a/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java index 8ba71e64597d..20b65f065bef 100644 --- a/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/io/druid/client/coordinator/CoordinatorClient.java @@ -28,7 +28,6 @@ import io.druid.discovery.DruidLeaderClient; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Interval; diff --git a/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java b/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java index 841cf576f721..70a0148c7aa5 100644 --- a/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java +++ b/server/src/main/java/io/druid/server/router/CoordinatorRuleManager.java @@ -35,7 +35,6 @@ import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.java.util.common.logger.Logger; import io.druid.server.coordinator.rules.Rule; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Duration; diff --git a/server/src/test/java/io/druid/discovery/DruidLeaderClientTest.java b/server/src/test/java/io/druid/discovery/DruidLeaderClientTest.java index cb8981989a29..8fdb412cd929 100644 --- a/server/src/test/java/io/druid/discovery/DruidLeaderClientTest.java +++ b/server/src/test/java/io/druid/discovery/DruidLeaderClientTest.java @@ -44,7 +44,6 @@ import io.druid.server.initialization.BaseJettyTest; import io.druid.server.initialization.ServerConfig; import io.druid.server.initialization.jetty.JettyServerInitializer; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.easymock.EasyMock; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java index 09e3e31dc1af..686d7399d57e 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java @@ -27,8 +27,6 @@ import io.druid.common.config.JacksonConfigManager; import io.druid.java.util.common.Intervals; import io.druid.server.coordinator.helper.DruidCoordinatorSegmentMerger; -import io.druid.server.security.AuthenticatorHttpClientWrapper; -import io.druid.server.security.AllowAllAuthenticator; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.LinearShardSpec; import org.easymock.EasyMock; diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index 467f7d9aa081..7cc0a9c5fd8e 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -43,8 +43,6 @@ import io.druid.server.DruidNode; import io.druid.server.coordinator.rules.IntervalLoadRule; import io.druid.server.coordinator.rules.Rule; -import io.druid.server.security.AuthenticatorHttpClientWrapper; -import io.druid.server.security.AllowAllAuthenticator; import io.druid.server.initialization.ServerConfig; import org.easymock.EasyMock; import org.joda.time.Interval; From eff666cbfae8666f2f7a1f4c10bf3045763d9350 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Wed, 13 Sep 2017 16:23:08 -0700 Subject: [PATCH 29/34] PR comments --- NOTICE | 6 + .../guice/annotations/EscalatedGlobal.java | 33 +++- .../druid/benchmark/query/SqlBenchmark.java | 20 +-- docs/content/configuration/auth.md | 29 ++-- .../extensions-core/druid-kerberos.md | 39 +++-- .../kerberos/KerberosAuthenticator.java | 24 ++- .../AuthenticationKerberosConfigTest.java | 58 ------- .../sql/QuantileSqlAggregatorTest.java | 30 +--- .../druid/indexing/kafka/KafkaIndexTask.java | 71 ++------ .../kafka/KafkaIndexTaskClientFactory.java | 10 +- .../kafka/supervisor/KafkaSupervisor.java | 7 +- .../kafka/supervisor/KafkaSupervisorSpec.java | 16 +- .../indexing/kafka/KafkaIndexTaskTest.java | 2 - .../kafka/supervisor/KafkaSupervisorTest.java | 12 +- .../io.druid.initialization.DruidModule | 2 +- .../overlord/RemoteTaskRunnerFactory.java | 12 +- .../overlord/http/OverlordResource.java | 10 +- .../security/SupervisorResourceFilter.java | 14 +- .../http/security/TaskResourceFilter.java | 11 +- .../supervisor/SupervisorResource.java | 33 ++-- .../overlord/http/OverlordResourceTest.java | 12 +- .../indexing/overlord/http/OverlordTest.java | 17 +- .../OverlordSecurityResourceFilterTest.java | 8 +- .../supervisor/SupervisorResourceTest.java | 21 +-- .../druid/testing/guice/DruidTestModule.java | 4 +- .../io/druid/client/BrokerServerView.java | 8 +- ...lteredHttpServerInventoryViewProvider.java | 12 +- .../druid/client/HttpServerInventoryView.java | 10 +- .../HttpServerInventoryViewProvider.java | 12 +- .../guice/CoordinatorDiscoveryModule.java | 10 +- .../guice/IndexingServiceDiscoveryModule.java | 10 +- .../guice/annotations/EscalatedClient.java | 32 +++- .../io/druid/guice/http/HttpClientModule.java | 51 +++++- .../guice/security/AuthenticatorModule.java | 2 - .../guice/security/AuthorizerModule.java | 2 - .../druid/initialization/Initialization.java | 3 + .../java/io/druid/server/QueryLifecycle.java | 61 ++++--- .../java/io/druid/server/QueryResource.java | 21 +-- .../http/security/AbstractResourceFilter.java | 9 -- .../http/security/ConfigResourceFilter.java | 15 +- .../security/DatasourceResourceFilter.java | 14 +- .../http/security/RulesResourceFilter.java | 14 +- .../http/security/StateResourceFilter.java | 15 +- .../AuthenticatorMapperModule.java | 14 +- .../AuthorizerMapperModule.java | 10 +- .../jetty/ForbiddenExceptionMapper.java | 44 +++++ .../jetty/JettyServerModule.java | 1 + .../cache/LookupCoordinatorManager.java | 12 +- .../io/druid/server/security/AuthConfig.java | 26 +-- .../druid/server/security/AuthTestUtils.java | 44 +++++ .../server/security/AuthenticationUtils.java | 5 +- .../druid/server/security/Authenticator.java | 9 +- .../server/security/AuthenticatorMapper.java | 22 --- .../server/security/AuthorizationUtils.java | 151 ++++-------------- .../io/druid/server/security/Authorizer.java | 3 +- .../server/security/AuthorizerMapper.java | 11 -- .../server/security/ForbiddenException.java | 37 +++++ .../PreResponseAuthorizationCheckFilter.java | 37 ++++- .../io/druid/server/security/Resource.java | 9 ++ .../security/UnsecuredResourceFilter.java | 4 - .../io/druid/client/BrokerServerViewTest.java | 3 - .../client/HttpServerInventoryViewTest.java | 3 +- .../firehose/EventReceiverFirehoseTest.java | 28 ++-- .../AsyncQueryForwardingServletTest.java | 14 +- .../io/druid/server/QueryResourceTest.java | 65 +++++--- .../server/http/DatasourcesResourceTest.java | 17 +- .../server/http/IntervalsResourceTest.java | 36 +++-- ...eResponseAuthorizationCheckFilterTest.java | 6 +- .../security/ResourceFilterTestHelper.java | 2 +- .../security/SecurityResourceFilterTest.java | 10 +- .../server/initialization/JettyQosTest.java | 14 +- .../server/initialization/JettyTest.java | 14 +- .../cache/LookupCoordinatorManagerTest.java | 62 +++---- .../main/java/io/druid/cli/CliOverlord.java | 3 +- .../src/main/java/io/druid/cli/CliRouter.java | 10 +- .../CoordinatorJettyServerInitializer.java | 9 +- .../MiddleManagerJettyServerInitializer.java | 3 +- .../cli/QueryJettyServerInitializer.java | 3 +- .../cli/RouterJettyServerInitializer.java | 3 +- .../java/io/druid/sql/avatica/DruidMeta.java | 5 +- .../sql/calcite/planner/DruidPlanner.java | 22 ++- .../sql/calcite/rel/DruidQueryBuilder.java | 2 +- .../sql/avatica/DruidAvaticaHandlerTest.java | 24 +-- .../druid/sql/avatica/DruidStatementTest.java | 21 +-- .../druid/sql/calcite/CalciteQueryTest.java | 25 +-- .../calcite/expression/ExpressionsTest.java | 12 +- .../sql/calcite/http/SqlResourceTest.java | 27 +--- .../sql/calcite/schema/DruidSchemaTest.java | 9 +- .../druid/sql/calcite/util/CalciteTests.java | 18 +-- 89 files changed, 756 insertions(+), 955 deletions(-) delete mode 100644 extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java create mode 100644 server/src/main/java/io/druid/server/initialization/jetty/ForbiddenExceptionMapper.java create mode 100644 server/src/main/java/io/druid/server/security/AuthTestUtils.java create mode 100644 server/src/main/java/io/druid/server/security/ForbiddenException.java diff --git a/NOTICE b/NOTICE index fb36a21373f4..71a02dd4c63f 100644 --- a/NOTICE +++ b/NOTICE @@ -77,3 +77,9 @@ This product contains a modified version of The Guava Authors's Closer class fro * https://github.com/google/guava * COMMIT TAG: * https://github.com/google/guava/blob/c462d69329709f72a17a64cb229d15e76e72199c + +This product contains code adapted from Apache Hadoop + * LICENSE: + * https://github.com/apache/hadoop/blob/trunk/LICENSE.txt (Apache License, Version 2.0) + * HOMEPAGE: + * http://hadoop.apache.org/ \ No newline at end of file diff --git a/api/src/main/java/io/druid/guice/annotations/EscalatedGlobal.java b/api/src/main/java/io/druid/guice/annotations/EscalatedGlobal.java index 01f80e305c6f..ef60239a4eb0 100644 --- a/api/src/main/java/io/druid/guice/annotations/EscalatedGlobal.java +++ b/api/src/main/java/io/druid/guice/annotations/EscalatedGlobal.java @@ -1,8 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package io.druid.guice.annotations; +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + /** - * Created by jw on 9/13/17. */ -public class EscalatedGlobal +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@PublicApi +public @interface EscalatedGlobal { } diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java index f25d0225ad22..76fd7473aec5 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SqlBenchmark.java @@ -39,13 +39,8 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.query.groupby.GroupByQuery; import io.druid.segment.QueryableIndex; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.planner.DruidPlanner; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.planner.PlannerFactory; @@ -73,7 +68,6 @@ import java.io.File; import java.util.ArrayList; import java.util.Arrays; -import java.util.Map; import java.util.concurrent.TimeUnit; /** @@ -120,8 +114,6 @@ public void setup() throws Exception final PlannerConfig plannerConfig = new PlannerConfig(); this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index); - final Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); plannerFactory = new PlannerFactory( CalciteTests.createMockSchema(walker, plannerConfig), CalciteTests.createMockQueryLifecycleFactory(walker), @@ -129,14 +121,8 @@ public void setup() throws Exception CalciteTests.createExprMacroTable(), plannerConfig, new AuthConfig(), - new AuthenticatorMapper(defaultMap, "allowAll"), - new AuthorizerMapper(null) { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }, + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ); groupByQuery = GroupByQuery diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index db4c86991418..29d32e502e55 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -6,9 +6,9 @@ layout: doc_page |Property|Type|Description|Default|Required| |--------|-----------|--------|--------|--------| -|`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|[]|no| -|`druid.auth.escalatedAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications|"allowAll"|no| -|`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |[]|no| +|`druid.auth.authenticationChain`|JSON List of Strings|List of Authenticator type names|["allowAll"]|no| +|`druid.auth.escalatedAuthenticator`|String|Type of the Authenticator that should be used for internal Druid communications. This Authenticator must be present in `druid.auth.authenticationChain`.|"allowAll"|no| +|`druid.auth.authorizers`|JSON List of Strings|List of Authorizer type names |["allowAll"]|no| ## Enabling Authentication/Authorization @@ -50,10 +50,7 @@ druid.auth.authorizers=["basic"] Only a single Authorizer will authorize any given request. -Druid includes two built in authorizers: - -### DenyAll Authorizer -The Authorizer with type name "denyAll" rejects all requests. +Druid includes one built in authorizer: ### AllowAll Authorizer The Authorizer with type name "allowAll" accepts all requests. @@ -74,17 +71,15 @@ An Authenticator implementation should provide some means through configuration ## Internal System User -Internal requests between Druid nodes (non-user initiated communications) need to have authentication credentials attached. These requests should be run as an "internal system user". - -We recommend that extension implementers follow the guidelines below regarding the "internal system user", for maximum compatibility between different Authenticator and Authorizer implementations. +Internal requests between Druid nodes (non-user initiated communications) need to have authentication credentials attached. -### Authorizer Internal System User Handling +These requests should be run as an "internal system user", an identity that represents the Druid cluster itself, with full access permissions. -Authorizers implementations must recognize and authorize an identity for the "internal system user", with unrestricted permissions. +The details of how the internal system user is defined is left to Authorizer and Authenticator implementations. -We recommend that this "internal system user" be represented by the identity string "__DRUID_INTERNAL_SYSTEM". This is a guideline only and not enforced; if an Authorizer needs to use a different identity string format, it is free to do so. +### Authorizer Internal System User Handling -Allowing the user to redefine what identity string represents the internal system user is also recommended. +Authorizers implementations must recognize and authorize an identity for the "internal system user", with full access permissions. ### Authenticator Internal System User Handling @@ -103,9 +98,3 @@ Authenticators must implement three methods related to the internal system user: `createEscalatedJettyClient` is similar to `createEscalatedClient`, except that it operates on a Jetty HttpClient. `createEscalatedAuthenticationResult` returns an AuthenticationResult containing the identity of the "internal system user". - -As with Authenticators, we recommend that the "internal system user" be represented by default with the identity string "__DRUID_INTERNAL_SYSTEM". This is a guideline and not enforced. - -We also recommend that Authenticator implementations allow the user to redefine the identity string used for the internal system users, if feasible. - - diff --git a/docs/content/development/extensions-core/druid-kerberos.md b/docs/content/development/extensions-core/druid-kerberos.md index 44e67cce2c7b..dc5a8fb5bb90 100644 --- a/docs/content/development/extensions-core/druid-kerberos.md +++ b/docs/content/development/extensions-core/druid-kerberos.md @@ -5,29 +5,46 @@ layout: doc_page # Druid-Kerberos Druid Extension to enable Authentication for Druid Nodes using Kerberos. -This extension adds AuthenticationFilter which is used to protect HTTP Endpoints using the simple and protected GSSAPI negotiation mechanism [SPNEGO](https://en.wikipedia.org/wiki/SPNEGO). +This extension adds an Authenticator which is used to protect HTTP Endpoints using the simple and protected GSSAPI negotiation mechanism [SPNEGO](https://en.wikipedia.org/wiki/SPNEGO). Make sure to [include](../../operations/including-extensions.html) `druid-kerberos` as an extension. ## Configuration +### Creating an Authenticator +``` +druid.auth.authenticatorChain=["MyKerberosAuthenticator"] + +druid.auth.authenticator.MyKerberosAuthenticator.type=kerberos +``` + +To use the Kerberos authenticator, add an authenticator with type `kerberos` to the authenticatorChain. The example above uses the name "MyKerberosAuthenticator" for the Authenticator. + +Configuration of the named authenticator is assigned through properties with the form: + +``` +druid.auth.authenticator.. +``` + +The configuration examples in the rest of this document will use "kerberos" as the name of the authenticator being configured. + +### Properties |Property|Possible Values|Description|Default|required| |--------|---------------|-----------|-------|--------| -|`druid.hadoop.security.kerberos.principal`|`druid@EXAMPLE.COM`| Principal user name, used for internal node communication|empty|Yes| -|`druid.hadoop.security.kerberos.keytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|empty|Yes| -|`druid.hadoop.security.spnego.principal`|`HTTP/_HOST@EXAMPLE.COM`| SPNego service principal used by druid nodes|empty|Yes| -|`druid.hadoop.security.spnego.keytab`|`/etc/security/keytabs/spnego.service.keytab`|SPNego service keytab used by druid nodes|empty|Yes| -|`druid.hadoop.security.spnego.authToLocal`|`RULE:[1:$1@$0](druid@EXAMPLE.COM)s/.*/druid DEFAULT`|It allows you to set a general rule for mapping principal names to local user names. It will be used if there is not an explicit mapping for the principal name that is being translated.|DEFAULT|No| -|`druid.hadoop.security.spnego.excludedPaths`|`['/status','/health']`| Array of HTTP paths which which does NOT need to be authenticated.|None|No| -|`druid.hadoop.security.spnego.cookieSignatureSecret`|`secretString`| Secret used to sign authentication cookies. It is advisable to explicitly set it, if you have multiple druid ndoes running on same machine with different ports as the Cookie Specification does not guarantee isolation by port.||No| +|`druid.auth.authenticator.kerberos.internalClientPrincipal`|`druid@EXAMPLE.COM`| Principal user name, used for internal node communication|empty|Yes| +|`druid.auth.authenticator.kerberos.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|empty|Yes| +|`druid.auth.authenticator.kerberos.serverPrincipal`|`HTTP/_HOST@EXAMPLE.COM`| SPNego service principal used by druid nodes|empty|Yes| +|`druid.auth.authenticator.kerberos.serverKeytab`|`/etc/security/keytabs/spnego.service.keytab`|SPNego service keytab used by druid nodes|empty|Yes| +|`druid.auth.authenticator.kerberos.authToLocal`|`RULE:[1:$1@$0](druid@EXAMPLE.COM)s/.*/druid DEFAULT`|It allows you to set a general rule for mapping principal names to local user names. It will be used if there is not an explicit mapping for the principal name that is being translated.|DEFAULT|No| +|`druid.auth.authenticator.kerberos.excludedPaths`|`['/status','/health']`| Array of HTTP paths which which does NOT need to be authenticated.|None|No| +|`druid.auth.authenticator.kerberos.cookieSignatureSecret`|`secretString`| Secret used to sign authentication cookies. It is advisable to explicitly set it, if you have multiple druid ndoes running on same machine with different ports as the Cookie Specification does not guarantee isolation by port.||No| +|`druid.auth.authenticator.kerberos.authorizerName`|`secretString`| Secret used to sign authentication cookies. It is advisable to explicitly set it, if you have multiple druid ndoes running on same machine with different ports as the Cookie Specification does not guarantee isolation by port.||No| As a note, it is required that the SPNego principal in use by the druid nodes must start with HTTP (This specified by [RFC-4559](https://tools.ietf.org/html/rfc4559)) and must be of the form "HTTP/_HOST@REALM". The special string _HOST will be replaced automatically with the value of config `druid.host` ### Auth to Local Syntax - - -`druid.hadoop.security.spnego.authToLocal` allows you to set a general rules for mapping principal names to local user names. +`druid.auth.authenticator.kerberos.authToLocal` allows you to set a general rules for mapping principal names to local user names. The syntax for mapping rules is `RULE:\[n:string](regexp)s/pattern/replacement/g`. The integer n indicates how many components the target principal should have. If this matches, then a string will be formed from string, substituting the realm of the principal for $0 and the n‘th component of the principal for $n. e.g. if the principal was druid/admin then `\[2:$2$1suffix]` would result in the string `admindruidsuffix`. If this string matches regexp, then the s//\[g] substitution command will be run over the string. The optional g will cause the substitution to be global over the string, instead of replacing only the first match in the string. If required, multiple rules can be be joined by newline character and specified as a String. diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java index 74a9549cfae0..6531cb4944d6 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/KerberosAuthenticator.java @@ -19,6 +19,7 @@ package io.druid.security.kerberos; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; @@ -81,6 +82,7 @@ import java.net.URI; import java.security.Principal; import java.security.PrivilegedExceptionAction; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -97,6 +99,7 @@ public class KerberosAuthenticator implements Authenticator { private static final Logger log = new Logger(KerberosAuthenticator.class); private static final Pattern HADOOP_AUTH_COOKIE_REGEX = Pattern.compile(".*p=(\\S+)&t=.*"); + public static final List DEFAULT_EXCLUDED_PATHS = Collections.emptyList(); private final DruidNode node; private final String serverPrincipal; @@ -109,6 +112,7 @@ public class KerberosAuthenticator implements Authenticator private final String authorizerName; private LoginContext loginContext; + @JsonCreator public KerberosAuthenticator( @JsonProperty("serverPrincipal") String serverPrincipal, @@ -119,7 +123,7 @@ public KerberosAuthenticator( @JsonProperty("excludedPaths") List excludedPaths, @JsonProperty("cookieSignatureSecret") String cookieSignatureSecret, @JsonProperty("authorizerName") String authorizerName, - @Self DruidNode node + @JacksonInject @Self DruidNode node ) { this.node = node; @@ -127,8 +131,8 @@ public KerberosAuthenticator( this.serverKeytab = serverKeytab; this.internalClientPrincipal = internalClientPrincipal; this.internalClientKeytab = internalClientKeytab; - this.authToLocal = authToLocal; - this.excludedPaths = excludedPaths; + this.authToLocal = authToLocal == null ? "DEFAULT" : authToLocal; + this.excludedPaths = excludedPaths == null ? DEFAULT_EXCLUDED_PATHS : excludedPaths; this.cookieSignatureSecret = cookieSignatureSecret; this.authorizerName = authorizerName; } @@ -186,6 +190,7 @@ public byte[][] getAllSecrets() } } + // Copied from hadoop-auth's AuthenticationFilter, to allow us to change error response handling in doFilterSuper @Override protected AuthenticationToken getToken(HttpServletRequest request) throws IOException, AuthenticationException { @@ -261,15 +266,18 @@ public void doFilter( clientPrincipal = null; } - request.setAttribute( - AuthConfig.DRUID_AUTHENTICATION_RESULT, - new AuthenticationResult(clientPrincipal, authorizerName) - ); + if (clientPrincipal != null) { + request.setAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT, + new AuthenticationResult(clientPrincipal, authorizerName) + ); + } } doFilterSuper(request, response, filterChain); } + // Copied from hadoop-auth's AuthenticationFilter, to allow us to change error response handling private void doFilterSuper(ServletRequest request, ServletResponse response, FilterChain filterChain) throws IOException, ServletException { @@ -518,7 +526,7 @@ private boolean isExcluded(String path) /** - * Kerberos context configuration for the JDK GSS library. + * Kerberos context configuration for the JDK GSS library. Copied from hadoop-auth's KerberosAuthenticationHandler. */ public static class DruidKerberosConfiguration extends Configuration { diff --git a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java b/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java deleted file mode 100644 index ac20101f11ee..000000000000 --- a/extensions-core/druid-kerberos/src/test/java/io/druid/security/kerberos/AuthenticationKerberosConfigTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.security.kerberos; - -import org.junit.Test; - -public class AuthenticationKerberosConfigTest -{ - @Test - public void testserde() - { - /* - Injector injector = Guice.createInjector( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.install(new PropertiesModule(Arrays.asList("test.runtime.properties"))); - binder.install(new ConfigModule()); - binder.install(new DruidGuiceExtensions()); - JsonConfigProvider.bind(binder, "druid.hadoop.security.kerberos", AuthenticationKerberosConfig.class); - } - - @Provides - @LazySingleton - public ObjectMapper jsonMapper() - { - return new DefaultObjectMapper(); - } - } - ); - - Properties props = injector.getInstance(Properties.class); - AuthenticationKerberosConfig config = injector.getInstance(AuthenticationKerberosConfig.class); - - Assert.assertEquals(props.getProperty("druid.hadoop.security.kerberos.principal"), config.getPrincipal()); - Assert.assertEquals(props.getProperty("druid.hadoop.security.kerberos.keytab"), config.getKeytab()); - */ - } -} diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index f4da8a11243a..d69334037a37 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.query.Druids; @@ -45,13 +44,8 @@ import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.virtual.ExpressionVirtualColumn; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -74,7 +68,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Map; public class QuantileSqlAggregatorTest { @@ -136,8 +129,7 @@ public void setUp() throws Exception ImmutableSet.of(new QuantileSqlAggregator()), ImmutableSet.of() ); - final Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); + plannerFactory = new PlannerFactory( druidSchema, CalciteTests.createMockQueryLifecycleFactory(walker), @@ -145,14 +137,8 @@ public void setUp() throws Exception CalciteTests.createExprMacroTable(), plannerConfig, new AuthConfig(), - new AuthenticatorMapper(defaultMap, "allowAll"), - new AuthorizerMapper(null) { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }, + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ); } @@ -241,9 +227,7 @@ public void testQuantileOnFloatAndLongs() throws Exception new QuantilePostAggregator("a7", "a5:agg", 0.999f), new QuantilePostAggregator("a8", "a8:agg", 0.50f) )) - .context(ImmutableMap.of( - "skipEmptyBuckets", true - )) + .context(ImmutableMap.of("skipEmptyBuckets", true)) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); @@ -303,9 +287,7 @@ public void testQuantileOnComplexColumn() throws Exception new QuantilePostAggregator("a5", "a5:agg", 0.999f), new QuantilePostAggregator("a6", "a4:agg", 0.999f) )) - .context(ImmutableMap.of( - "skipEmptyBuckets", true - )) + .context(ImmutableMap.of("skipEmptyBuckets", true)) .build(), Iterables.getOnlyElement(queryLogHook.getRecordedQueries()) ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 8f583909f4b4..08965c021644 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -80,9 +80,9 @@ import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; @@ -144,7 +144,6 @@ public enum Status private final InputRowParser parser; private final KafkaTuningConfig tuningConfig; private final KafkaIOConfig ioConfig; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; private final Optional chatHandlerProvider; @@ -211,7 +210,6 @@ public KafkaIndexTask( @JsonProperty("ioConfig") KafkaIOConfig ioConfig, @JsonProperty("context") Map context, @JacksonInject ChatHandlerProvider chatHandlerProvider, - @JacksonInject AuthConfig authConfig, @JacksonInject AuthorizerMapper authorizerMapper ) { @@ -228,7 +226,6 @@ public KafkaIndexTask( this.tuningConfig = Preconditions.checkNotNull(tuningConfig, "tuningConfig"); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider); - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; this.endOffsets.putAll(ioConfig.getEndPartitions().getPartitionOffsetMap()); @@ -655,7 +652,12 @@ private Access authorizationCheck(final HttpServletRequest req, Action action) action ); - return AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + Access access = AuthorizationUtils.authorizeResourceAction(req, resourceAction, authorizerMapper); + if (!access.isAllowed()) { + throw new ForbiddenException(access.toString()); + } + + return access; } @Override @@ -727,13 +729,7 @@ public Sequence run(final QueryPlus queryPlus, final Map r @Path("/stop") public Response stop(@Context final HttpServletRequest req) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } - + authorizationCheck(req, Action.WRITE); stopGracefully(); return Response.status(Response.Status.OK).build(); } @@ -743,12 +739,7 @@ public Response stop(@Context final HttpServletRequest req) @Produces(MediaType.APPLICATION_JSON) public Status getStatusHTTP(@Context final HttpServletRequest req) { - Access access = authorizationCheck(req, Action.READ); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } - + authorizationCheck(req, Action.READ); return status; } @@ -762,12 +753,7 @@ public Status getStatus() @Produces(MediaType.APPLICATION_JSON) public Map getCurrentOffsets(@Context final HttpServletRequest req) { - Access access = authorizationCheck(req, Action.READ); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } - + authorizationCheck(req, Action.READ); return getCurrentOffsets(); } @@ -781,12 +767,7 @@ public Map getCurrentOffsets() @Produces(MediaType.APPLICATION_JSON) public Map getEndOffsetsHTTP(@Context final HttpServletRequest req) { - Access access = authorizationCheck(req, Action.READ); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } - + authorizationCheck(req, Action.READ); return getEndOffsets(); } @@ -805,13 +786,7 @@ public Response setEndOffsetsHTTP( @Context final HttpServletRequest req ) throws InterruptedException { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } - + authorizationCheck(req, Action.WRITE); return setEndOffsets(offsets, resume); } @@ -888,13 +863,7 @@ public Response pauseHTTP( @Context final HttpServletRequest req ) throws InterruptedException { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } - + authorizationCheck(req, Action.WRITE); return pause(timeout); } @@ -949,13 +918,7 @@ public Response pause(final long timeout) throws InterruptedException @Path("/resume") public Response resumeHTTP(@Context final HttpServletRequest req) throws InterruptedException { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .entity("Request authorization failed.") - .build(); - } - + authorizationCheck(req, Action.WRITE); resume(); return Response.status(Response.Status.OK).build(); } @@ -985,11 +948,7 @@ public void resume() throws InterruptedException @Produces(MediaType.APPLICATION_JSON) public DateTime getStartTime(@Context final HttpServletRequest req) { - Access access = authorizationCheck(req, Action.WRITE); - if (!access.isAllowed()) { - log.warn("Authorization failure."); - return null; - } + authorizationCheck(req, Action.WRITE); return startTime; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java index 41bb5602e45f..869392642003 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTaskClientFactory.java @@ -22,10 +22,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.metamx.http.client.HttpClient; -import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Json; import io.druid.indexing.common.TaskInfoProvider; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.joda.time.Duration; public class KafkaIndexTaskClientFactory @@ -35,12 +34,11 @@ public class KafkaIndexTaskClientFactory @Inject public KafkaIndexTaskClientFactory( - @Global HttpClient httpClient, - @Json ObjectMapper mapper, - AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + @EscalatedGlobal HttpClient httpClient, + @Json ObjectMapper mapper ) { - this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); + this.httpClient = httpClient; this.mapper = mapper; } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java index dac983eaab77..f2065e1a4c80 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -71,7 +71,6 @@ import io.druid.java.util.common.StringUtils; import io.druid.metadata.EntryExistsException; import io.druid.server.metrics.DruidMonitorSchedulerConfig; -import io.druid.server.security.AuthConfig; import org.apache.commons.codec.digest.DigestUtils; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; @@ -195,7 +194,6 @@ private static class TaskData private final KafkaSupervisorSpec spec; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final AuthConfig authConfig; private final String dataSource; private final KafkaSupervisorIOConfig ioConfig; private final KafkaSupervisorTuningConfig tuningConfig; @@ -229,8 +227,7 @@ public KafkaSupervisor( final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, final KafkaIndexTaskClientFactory taskClientFactory, final ObjectMapper mapper, - final KafkaSupervisorSpec spec, - final AuthConfig authConfig + final KafkaSupervisorSpec spec ) { this.taskStorage = taskStorage; @@ -240,7 +237,6 @@ public KafkaSupervisor( this.spec = spec; this.emitter = spec.getEmitter(); this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); - this.authConfig = spec.getAuthConfig(); this.dataSource = spec.getDataSchema().getDataSource(); this.ioConfig = spec.getIoConfig(); @@ -1437,7 +1433,6 @@ private void createKafkaTasksForGroup(int groupId, int replicas) kafkaIOConfig, spec.getContext(), null, - new AuthConfig(), null ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 909cfa59041e..559428fc2e49 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -35,7 +35,6 @@ import io.druid.indexing.overlord.supervisor.SupervisorSpec; import io.druid.segment.indexing.DataSchema; import io.druid.server.metrics.DruidMonitorSchedulerConfig; -import io.druid.server.security.AuthConfig; import java.util.List; import java.util.Map; @@ -54,7 +53,6 @@ public class KafkaSupervisorSpec implements SupervisorSpec private final ObjectMapper mapper; private final ServiceEmitter emitter; private final DruidMonitorSchedulerConfig monitorSchedulerConfig; - private final AuthConfig authConfig; @JsonCreator public KafkaSupervisorSpec( @@ -68,9 +66,8 @@ public KafkaSupervisorSpec( @JacksonInject KafkaIndexTaskClientFactory kafkaIndexTaskClientFactory, @JacksonInject @Json ObjectMapper mapper, @JacksonInject ServiceEmitter emitter, - @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, - @JacksonInject AuthConfig authConfig - ) + @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig + ) { this.dataSchema = Preconditions.checkNotNull(dataSchema, "dataSchema"); this.tuningConfig = tuningConfig != null @@ -103,7 +100,6 @@ public KafkaSupervisorSpec( this.mapper = mapper; this.emitter = emitter; this.monitorSchedulerConfig = monitorSchedulerConfig; - this.authConfig = authConfig; } @JsonProperty @@ -146,11 +142,6 @@ public DruidMonitorSchedulerConfig getMonitorSchedulerConfig() return monitorSchedulerConfig; } - public AuthConfig getAuthConfig() - { - return authConfig; - } - @Override public Supervisor createSupervisor() { @@ -160,8 +151,7 @@ public Supervisor createSupervisor() indexerMetadataStorageCoordinator, kafkaIndexTaskClientFactory, mapper, - this, - authConfig + this ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index e7b71b3deda9..453405ce08ee 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -125,7 +125,6 @@ import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.server.coordination.ServerType; -import io.druid.server.security.AuthConfig; import io.druid.timeline.DataSegment; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; @@ -1489,7 +1488,6 @@ private KafkaIndexTask createTask( ioConfig, null, null, - new AuthConfig(), null ); task.setPollRetryMs(POLL_RETRY_MS); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 54d137fc2329..672f0c92bb32 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -69,9 +69,6 @@ import io.druid.segment.realtime.FireDepartment; import io.druid.server.metrics.DruidMonitorSchedulerConfig; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthenticatorHttpClientWrapper; -import io.druid.server.security.AllowAllAuthenticator; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -1824,8 +1821,7 @@ private KafkaSupervisor getSupervisor( KafkaIndexTaskClientFactory taskClientFactory = new KafkaIndexTaskClientFactory( null, - null, - new AuthenticatorHttpClientWrapper(new AllowAllAuthenticator()) + null ) { @Override @@ -1861,8 +1857,7 @@ public KafkaIndexTaskClient build( taskClientFactory, objectMapper, new NoopServiceEmitter(), - new DruidMonitorSchedulerConfig(), - new AuthConfig() + new DruidMonitorSchedulerConfig() ) ); } @@ -1929,7 +1924,6 @@ private KafkaIndexTask createKafkaIndexTask( ), ImmutableMap.of(), null, - new AuthConfig(), null ); } @@ -1963,7 +1957,7 @@ public TestableKafkaSupervisor( KafkaSupervisorSpec spec ) { - super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, mapper, spec, new AuthConfig()); + super(taskStorage, taskMaster, indexerMetadataStorageCoordinator, taskClientFactory, mapper, spec); } @Override diff --git a/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index ee274af1dd2f..cf8371810e4f 100644 --- a/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/extensions-core/mysql-metadata-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -1 +1 @@ -io.druid.metadata.storage.mysql.MySQLMetadataStorageModule \ No newline at end of file +io.druid.metadata.storage.mysql.MySQLMetadataStorageModule diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java index 727da42bf54a..5e0b165c0cab 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java @@ -24,14 +24,13 @@ import com.google.inject.Inject; import com.metamx.http.client.HttpClient; import io.druid.curator.cache.PathChildrenCacheFactory; -import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.indexing.overlord.autoscaling.NoopProvisioningStrategy; import io.druid.indexing.overlord.autoscaling.ProvisioningSchedulerConfig; import io.druid.indexing.overlord.autoscaling.ProvisioningStrategy; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.server.initialization.IndexerZkConfig; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.apache.curator.framework.CuratorFramework; /** @@ -54,18 +53,17 @@ public RemoteTaskRunnerFactory( final RemoteTaskRunnerConfig remoteTaskRunnerConfig, final IndexerZkConfig zkPaths, final ObjectMapper jsonMapper, - @Global final HttpClient httpClient, + @EscalatedGlobal final HttpClient httpClient, final Supplier workerConfigRef, final ProvisioningSchedulerConfig provisioningSchedulerConfig, - final ProvisioningStrategy provisioningStrategy, - final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper - ) + final ProvisioningStrategy provisioningStrategy + ) { this.curator = curator; this.remoteTaskRunnerConfig = remoteTaskRunnerConfig; this.zkPaths = zkPaths; this.jsonMapper = jsonMapper; - this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); + this.httpClient = httpClient; this.workerConfigRef = workerConfigRef; this.provisioningSchedulerConfig = provisioningSchedulerConfig; this.provisioningStrategy = provisioningStrategy; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 51e59e4f3d37..d8ef4bc87019 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -57,9 +57,9 @@ import io.druid.server.http.security.StateResourceFilter; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; @@ -101,7 +101,6 @@ public class OverlordResource private final TaskLogStreamer taskLogStreamer; private final JacksonConfigManager configManager; private final AuditManager auditManager; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; private AtomicReference workerConfigRef = null; @@ -113,7 +112,6 @@ public OverlordResource( TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager, AuditManager auditManager, - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) throws Exception { @@ -122,7 +120,6 @@ public OverlordResource( this.taskLogStreamer = taskLogStreamer; this.configManager = configManager; this.auditManager = auditManager; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; } @@ -148,10 +145,7 @@ public Response taskPost( ); if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN) - .header("Access-Check-Result", authResult) - .entity(ImmutableMap.of("error", "Not authorized.")) - .build(); + throw new ForbiddenException(authResult.toString()); } return asLeaderWith( diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java index d8cdbf12eec9..0af8490b3b15 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/SupervisorResourceFilter.java @@ -33,9 +33,9 @@ import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.ResourceAction; import javax.ws.rs.WebApplicationException; @@ -49,12 +49,11 @@ public class SupervisorResourceFilter extends AbstractResourceFilter @Inject public SupervisorResourceFilter( - AuthConfig authConfig, AuthorizerMapper authorizerMapper, SupervisorManager supervisorManager ) { - super(authConfig, authorizerMapper); + super(authorizerMapper); this.supervisorManager = supervisorManager; } @@ -100,17 +99,12 @@ public boolean apply(PathSegment input) Access authResult = AuthorizationUtils.authorizeAllResourceActions( getReq(), - spec.getDataSources(), - resourceActionFunction, + Iterables.transform(spec.getDataSources(), resourceActionFunction), getAuthorizerMapper() ); if (!authResult.isAllowed()) { - throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) - .entity( - StringUtils.format("Access-Check-Result: %s", authResult.toString()) - ) - .build()); + throw new ForbiddenException(authResult.toString()); } return request; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java index e70d3006af42..037720547a2f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/security/TaskResourceFilter.java @@ -31,9 +31,9 @@ import io.druid.java.util.common.StringUtils; import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; @@ -56,11 +56,10 @@ public class TaskResourceFilter extends AbstractResourceFilter @Inject public TaskResourceFilter( TaskStorageQueryAdapter taskStorageQueryAdapter, - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizerMapper); + super(authorizerMapper); this.taskStorageQueryAdapter = taskStorageQueryAdapter; } @@ -106,11 +105,7 @@ public boolean apply(PathSegment input) ); if (!authResult.isAllowed()) { - throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN) - .entity( - StringUtils.format("Access-Check-Result: %s", authResult.toString()) - ) - .build()); + throw new ForbiddenException(authResult.toString()); } return request; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java index 6dac0cf0c0c4..ec3aad39bf06 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -36,6 +37,7 @@ import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -92,13 +94,12 @@ public Response apply(SupervisorManager manager) Access authResult = AuthorizationUtils.authorizeAllResourceActions( req, - spec.getDataSources(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + Iterables.transform(spec.getDataSources(), AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR), authorizerMapper ); if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); + throw new ForbiddenException(authResult.toString()); } manager.createOrUpdateAndStartSupervisor(spec); @@ -125,8 +126,10 @@ public Response apply(final SupervisorManager manager) if (supervisorSpecOptional.isPresent()) { Access accessResult = AuthorizationUtils.authorizeAllResourceActions( req, - supervisorSpecOptional.get().getDataSources(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + Iterables.transform( + supervisorSpecOptional.get().getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR + ), authorizerMapper ); @@ -136,12 +139,14 @@ public Response apply(final SupervisorManager manager) } } - AuthorizationUtils.authorizeAllResourceActions( - req, - Lists.newArrayList(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, - authorizerMapper - ); + // If there were no supervisorIds, go ahead and authorize the request. + if (manager.getSupervisorIds().size() == 0) { + AuthorizationUtils.authorizeAllResourceActions( + req, + Lists.newArrayList(), + authorizerMapper + ); + } return Response.ok(supervisorIds).build(); } @@ -248,8 +253,10 @@ public boolean apply(String id) } Access accessResult = AuthorizationUtils.authorizeAllResourceActions( req, - supervisorSpecOptional.get().getDataSources(), - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + Iterables.transform( + supervisorSpecOptional.get().getDataSources(), + AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR + ), authorizerMapper ); return accessResult.isAllowed(); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 882f4125c3f9..0de16715d2c9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -42,12 +42,15 @@ import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; @@ -62,6 +65,9 @@ public class OverlordResourceTest private HttpServletRequest req; private TaskRunner taskRunner; + @Rule + public ExpectedException expectedException = ExpectedException.none(); + @Before public void setUp() throws Exception { @@ -100,7 +106,6 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso null, null, null, - new AuthConfig(true, null, null, null), authMapper ); } @@ -249,12 +254,13 @@ public void testSecuredGetRunningTasks() @Test public void testSecuredTaskPost() { + expectedException.expect(ForbiddenException.class); + expectedException.expectMessage("Allowed:false, Message:"); expectAuthorizationTokenCheck(); EasyMock.replay(taskRunner, taskMaster, tsqa, req); Task task = NoopTask.create(); - Response response = overlordResource.taskPost(task, req); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), response.getStatus()); + overlordResource.taskPost(task, req); } @After diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index f0bbe31a0feb..fd4f13455d55 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -59,11 +59,9 @@ import io.druid.server.initialization.ServerConfig; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthTestUtils; import io.druid.server.security.AuthenticationResult; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryOneTime; @@ -131,6 +129,7 @@ private void tearDownServerAndCurator() public void setUp() throws Exception { req = EasyMock.createMock(HttpServletRequest.class); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).anyTimes(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( new AuthenticationResult("druid", "druid") ).anyTimes(); @@ -213,15 +212,6 @@ public void testOverlordRun() throws Exception } Assert.assertEquals(taskMaster.getCurrentLeader(), druidNode.getHostAndPort()); - AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { - - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }; - // Test Overlord resource stuff overlordResource = new OverlordResource( taskMaster, @@ -229,8 +219,7 @@ public Authorizer getAuthorizer(String name) null, null, null, - new AuthConfig(), - authorizerMapper + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); Response response = overlordResource.getLeader(); Assert.assertEquals(druidNode.getHostAndPort(), response.getEntity()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java index 2f3cdb0d7c8b..8ac21215f2e3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/security/OverlordSecurityResourceFilterTest.java @@ -36,6 +36,7 @@ import io.druid.server.http.security.AbstractResourceFilter; import io.druid.server.http.security.ResourceFilterTestHelper; import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.ForbiddenException; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -44,8 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; import java.util.Collection; import java.util.List; @@ -155,7 +154,7 @@ public void testResourcesFilteringAccess() Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); } - @Test(expected = WebApplicationException.class) + @Test(expected = ForbiddenException.class) public void testDatasourcesResourcesFilteringNoAccess() { setUpMockExpectations(requestPath, false, requestMethod); @@ -165,8 +164,7 @@ public void testDatasourcesResourcesFilteringNoAccess() try { resourceFilter.getRequestFilter().filter(request); } - catch (WebApplicationException e) { - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), e.getResponse().getStatus()); + catch (ForbiddenException e) { throw e; } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 0411d216ea66..74476d9ff517 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -27,12 +27,10 @@ import com.google.common.collect.Maps; import io.druid.indexing.overlord.DataSourceMetadata; import io.druid.indexing.overlord.TaskMaster; -import io.druid.server.security.AllowAllAuthorizer; -import io.druid.server.security.AuthenticationResult; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; import io.druid.java.util.common.DateTimes; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthTestUtils; +import io.druid.server.security.AuthenticationResult; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -66,15 +64,7 @@ public class SupervisorResourceTest extends EasyMockSupport @Before public void setUp() throws Exception { - AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { - - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }; - supervisorResource = new SupervisorResource(taskMaster, new AuthConfig(), authorizerMapper); + supervisorResource = new SupervisorResource(taskMaster, new AuthConfig(), AuthTestUtils.TEST_AUTHORIZER_MAPPER); } @Test @@ -91,6 +81,7 @@ public List getDataSources() EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)); EasyMock.expect(supervisorManager.createOrUpdateAndStartSupervisor(spec)).andReturn(true); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce(); EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( new AuthenticationResult("druid", "druid") ).atLeastOnce(); @@ -136,9 +127,10 @@ public List getDataSources() }; EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)); - EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds); + EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds).atLeastOnce(); EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)); EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce(); EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( new AuthenticationResult("druid", "druid") ).atLeastOnce(); @@ -287,6 +279,7 @@ public List getDataSources() }; EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).atLeastOnce(); EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).atLeastOnce(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce(); EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( new AuthenticationResult("druid", "druid") ).atLeastOnce(); diff --git a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java index 14b48da5f7d5..3d210c5e8204 100644 --- a/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java +++ b/integration-tests/src/main/java/io/druid/testing/guice/DruidTestModule.java @@ -34,7 +34,7 @@ import io.druid.curator.CuratorConfig; import io.druid.guice.JsonConfigProvider; import io.druid.guice.ManageLifecycle; -import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.EscalatedClient; import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; @@ -67,7 +67,7 @@ public void configure(Binder binder) public HttpClient getHttpClient( IntegrationTestingConfig config, Lifecycle lifecycle, - @Client HttpClient delegate + @EscalatedClient HttpClient delegate ) throws Exception { diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index 3061f686c2c0..fe0e2d36567d 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -31,7 +31,7 @@ import io.druid.client.selector.ServerSelector; import io.druid.client.selector.TierSelectorStrategy; import io.druid.concurrent.Execs; -import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.EscalatedClient; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; @@ -40,7 +40,6 @@ import io.druid.query.QueryToolChestWarehouse; import io.druid.query.QueryWatcher; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; import io.druid.timeline.partition.PartitionChunk; @@ -83,8 +82,7 @@ public BrokerServerView( QueryToolChestWarehouse warehouse, QueryWatcher queryWatcher, @Smile ObjectMapper smileMapper, - @Client HttpClient httpClient, - AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper, + @EscalatedClient HttpClient httpClient, FilteredServerInventoryView baseView, TierSelectorStrategy tierSelectorStrategy, ServiceEmitter emitter, @@ -94,7 +92,7 @@ public BrokerServerView( this.warehouse = warehouse; this.queryWatcher = queryWatcher; this.smileMapper = smileMapper; - this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); + this.httpClient = httpClient; this.baseView = baseView; this.tierSelectorStrategy = tierSelectorStrategy; this.emitter = emitter; diff --git a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java index 64278007b601..54108b951547 100644 --- a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java @@ -24,11 +24,10 @@ import com.google.common.base.Predicates; import com.metamx.http.client.HttpClient; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.EscalatedClient; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Pair; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import javax.validation.constraints.NotNull; @@ -39,7 +38,7 @@ public class FilteredHttpServerInventoryViewProvider implements FilteredServerIn { @JacksonInject @NotNull - @Client + @EscalatedClient HttpClient httpClient = null; @JacksonInject @@ -55,10 +54,6 @@ public class FilteredHttpServerInventoryViewProvider implements FilteredServerIn @NotNull private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = null; - @JacksonInject - @NotNull - private AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = null; - @Override public HttpServerInventoryView get() { @@ -66,8 +61,7 @@ public HttpServerInventoryView get() smileMapper, httpClient, druidNodeDiscoveryProvider, Predicates.>alwaysTrue(), - config, - authenticatorHttpClientWrapper + config ); } } diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryView.java b/server/src/main/java/io/druid/client/HttpServerInventoryView.java index d25a50e55af0..a9fa4bf7f78e 100644 --- a/server/src/main/java/io/druid/client/HttpServerInventoryView.java +++ b/server/src/main/java/io/druid/client/HttpServerInventoryView.java @@ -44,7 +44,7 @@ import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; @@ -58,7 +58,6 @@ import io.druid.server.coordination.SegmentChangeRequestHistory; import io.druid.server.coordination.SegmentChangeRequestLoad; import io.druid.server.coordination.SegmentChangeRequestsSnapshot; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -120,14 +119,13 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer @Inject public HttpServerInventoryView( final @Smile ObjectMapper smileMapper, - final @Global HttpClient httpClient, + final @EscalatedGlobal HttpClient httpClient, final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, final Predicate> defaultFilter, - final HttpServerInventoryViewConfig config, - final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + final HttpServerInventoryViewConfig config ) { - this.httpClient = authenticatorHttpClientWrapper.getEscalatedClient(httpClient); + this.httpClient = httpClient; this.smileMapper = smileMapper; this.druidNodeDiscoveryProvider = druidNodeDiscoveryProvider; this.defaultFilter = defaultFilter; diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java index ed2bca0a8808..bfcb3b262384 100644 --- a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java @@ -24,11 +24,10 @@ import com.google.common.base.Predicates; import com.metamx.http.client.HttpClient; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.EscalatedClient; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Pair; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import io.druid.timeline.DataSegment; import javax.validation.constraints.NotNull; @@ -39,7 +38,7 @@ public class HttpServerInventoryViewProvider implements ServerInventoryViewProvi { @JacksonInject @NotNull - @Client + @EscalatedClient HttpClient httpClient = null; @JacksonInject @@ -55,10 +54,6 @@ public class HttpServerInventoryViewProvider implements ServerInventoryViewProvi @NotNull private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = null; - @JacksonInject - @NotNull - private AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = null; - @Override public HttpServerInventoryView get() { @@ -67,8 +62,7 @@ public HttpServerInventoryView get() httpClient, druidNodeDiscoveryProvider, Predicates.>alwaysTrue(), - config, - authenticatorHttpClientWrapper + config ); } } diff --git a/server/src/main/java/io/druid/guice/CoordinatorDiscoveryModule.java b/server/src/main/java/io/druid/guice/CoordinatorDiscoveryModule.java index b72afd8b2aca..6af0f2d02250 100644 --- a/server/src/main/java/io/druid/guice/CoordinatorDiscoveryModule.java +++ b/server/src/main/java/io/druid/guice/CoordinatorDiscoveryModule.java @@ -29,8 +29,7 @@ import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.discovery.DruidLeaderClient; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.annotations.Global; -import io.druid.server.security.AuthenticatorHttpClientWrapper; +import io.druid.guice.annotations.EscalatedGlobal; /** */ @@ -57,14 +56,13 @@ public ServerDiscoverySelector getServiceProvider( @Coordinator @ManageLifecycle public DruidLeaderClient getLeaderHttpClient( - @Global HttpClient httpClient, + @EscalatedGlobal HttpClient httpClient, DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, - @Coordinator ServerDiscoverySelector serverDiscoverySelector, - AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + @Coordinator ServerDiscoverySelector serverDiscoverySelector ) { return new DruidLeaderClient( - authenticatorHttpClientWrapper.getEscalatedClient(httpClient), + httpClient, druidNodeDiscoveryProvider, DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, "/druid/coordinator/v1/leader", diff --git a/server/src/main/java/io/druid/guice/IndexingServiceDiscoveryModule.java b/server/src/main/java/io/druid/guice/IndexingServiceDiscoveryModule.java index 2b56620b1404..05da8f301bb4 100644 --- a/server/src/main/java/io/druid/guice/IndexingServiceDiscoveryModule.java +++ b/server/src/main/java/io/druid/guice/IndexingServiceDiscoveryModule.java @@ -29,8 +29,7 @@ import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.discovery.DruidLeaderClient; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.annotations.Global; -import io.druid.server.security.AuthenticatorHttpClientWrapper; +import io.druid.guice.annotations.EscalatedGlobal; /** */ @@ -57,14 +56,13 @@ public ServerDiscoverySelector getServiceProvider( @IndexingService @ManageLifecycle public DruidLeaderClient getLeaderHttpClient( - @Global HttpClient httpClient, + @EscalatedGlobal HttpClient httpClient, DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, - @IndexingService ServerDiscoverySelector serverDiscoverySelector, - AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + @IndexingService ServerDiscoverySelector serverDiscoverySelector ) { return new DruidLeaderClient( - authenticatorHttpClientWrapper.getEscalatedClient(httpClient), + httpClient, druidNodeDiscoveryProvider, DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, "/druid/indexer/v1/leader", diff --git a/server/src/main/java/io/druid/guice/annotations/EscalatedClient.java b/server/src/main/java/io/druid/guice/annotations/EscalatedClient.java index ff048075f2a3..161ad51fc495 100644 --- a/server/src/main/java/io/druid/guice/annotations/EscalatedClient.java +++ b/server/src/main/java/io/druid/guice/annotations/EscalatedClient.java @@ -1,8 +1,36 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package io.druid.guice.annotations; +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + /** - * Created by jw on 9/13/17. */ -public class EscalatedClient +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface EscalatedClient { } diff --git a/server/src/main/java/io/druid/guice/http/HttpClientModule.java b/server/src/main/java/io/druid/guice/http/HttpClientModule.java index 3eb38fd892e9..848bb9b3eb8f 100644 --- a/server/src/main/java/io/druid/guice/http/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/http/HttpClientModule.java @@ -19,17 +19,24 @@ package io.druid.guice.http; +import com.google.common.collect.Sets; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Module; import com.metamx.http.client.HttpClient; import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.EscalatedClient; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Global; import io.druid.java.util.common.StringUtils; +import io.druid.server.security.Authenticator; +import io.druid.server.security.AuthenticatorMapper; import java.lang.annotation.Annotation; +import java.util.Set; /** */ @@ -40,9 +47,19 @@ public static HttpClientModule global() return new HttpClientModule("druid.global.http", Global.class); } + public static HttpClientModule escalatedGlobal() + { + return new HttpClientModule("druid.global.http", EscalatedGlobal.class); + } + + private static Set> ESCALATING_ANNOTATIONS = Sets.newHashSet( + EscalatedGlobal.class, EscalatedClient.class + ); + private final String propertyPrefix; private Annotation annotation = null; private Class annotationClazz = null; + private boolean isEscalated = false; public HttpClientModule(String propertyPrefix) { @@ -53,6 +70,8 @@ public HttpClientModule(String propertyPrefix, Class annot { this.propertyPrefix = propertyPrefix; this.annotationClazz = annotation; + + isEscalated = ESCALATING_ANNOTATIONS.contains(annotationClazz); } public HttpClientModule(String propertyPrefix, Annotation annotation) @@ -68,36 +87,48 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotation); binder.bind(HttpClient.class) .annotatedWith(annotation) - .toProvider(new HttpClientProvider(annotation)) + .toProvider(new HttpClientProvider(annotation, isEscalated)) .in(LazySingleton.class); } else if (annotationClazz != null) { JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotationClazz); binder.bind(HttpClient.class) .annotatedWith(annotationClazz) - .toProvider(new HttpClientProvider(annotationClazz)) + .toProvider(new HttpClientProvider(annotationClazz, isEscalated)) .in(LazySingleton.class); } else { JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class); binder.bind(HttpClient.class) - .toProvider(new HttpClientProvider()) + .toProvider(new HttpClientProvider(isEscalated)) .in(LazySingleton.class); } } public static class HttpClientProvider extends AbstractHttpClientProvider { - public HttpClientProvider() + private boolean isEscalated; + private Authenticator escalatingAuthenticator; + + public HttpClientProvider(boolean isEscalated) { + this.isEscalated = isEscalated; } - public HttpClientProvider(Annotation annotation) + public HttpClientProvider(Annotation annotation, boolean isEscalated) { super(annotation); + this.isEscalated = isEscalated; } - public HttpClientProvider(Class annotationClazz) + public HttpClientProvider(Class annotationClazz, boolean isEscalated) { super(annotationClazz); + this.isEscalated = isEscalated; + } + + @Inject + public void inject(AuthenticatorMapper authenticatorMapper) + { + this.escalatingAuthenticator = authenticatorMapper.getEscalatingAuthenticator(); } @Override @@ -118,10 +149,16 @@ public HttpClient get() builder.withSslContext(getSslContextBinding().getProvider().get()); } - return HttpClientInit.createClient( + HttpClient client = HttpClientInit.createClient( builder.build(), LifecycleUtils.asMmxLifecycle(getLifecycleProvider().get()) ); + + if (isEscalated) { + return escalatingAuthenticator.createEscalatedClient(client); + } else { + return client; + } } } } diff --git a/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java b/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java index 15a91f35debb..7c09284ef52f 100644 --- a/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java +++ b/server/src/main/java/io/druid/guice/security/AuthenticatorModule.java @@ -26,7 +26,6 @@ import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; import io.druid.guice.LazySingleton; -import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; import io.druid.server.security.Authenticator; import io.druid.server.security.AllowAllAuthenticator; @@ -44,7 +43,6 @@ public void configure(Binder binder) } @Provides - @ManageLifecycle @Named("allowAll") public Authenticator getAuthenticator() { diff --git a/server/src/main/java/io/druid/guice/security/AuthorizerModule.java b/server/src/main/java/io/druid/guice/security/AuthorizerModule.java index 4de3eb9958ba..a39a3dc09462 100644 --- a/server/src/main/java/io/druid/guice/security/AuthorizerModule.java +++ b/server/src/main/java/io/druid/guice/security/AuthorizerModule.java @@ -26,7 +26,6 @@ import com.google.inject.multibindings.MapBinder; import com.google.inject.name.Named; import io.druid.guice.LazySingleton; -import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; import io.druid.server.security.Authorizer; import io.druid.server.security.AllowAllAuthorizer; @@ -44,7 +43,6 @@ public void configure(Binder binder) } @Provides - @ManageLifecycle @Named("allowAll") public Authorizer getAuthorizer() { diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 2850127e3a92..cf4eb3a55c01 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -53,6 +53,7 @@ import io.druid.guice.StartupLoggingModule; import io.druid.guice.StorageNodeModule; import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.EscalatedClient; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.guice.http.HttpClientModule; @@ -352,7 +353,9 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new LifecycleModule(), EmitterModule.class, HttpClientModule.global(), + HttpClientModule.escalatedGlobal(), new HttpClientModule("druid.broker.http", Client.class), + new HttpClientModule("druid.broker.http", EscalatedClient.class), new CuratorModule(), new AnnouncerModule(), new AWSModule(), diff --git a/server/src/main/java/io/druid/server/QueryLifecycle.java b/server/src/main/java/io/druid/server/QueryLifecycle.java index 87d55eeabb8f..a892c06421c2 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycle.java +++ b/server/src/main/java/io/druid/server/QueryLifecycle.java @@ -20,6 +20,7 @@ package io.druid.server; import com.google.common.base.Strings; +import com.google.common.collect.Iterables; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.DirectDruidClient; import io.druid.java.util.common.DateTimes; @@ -132,7 +133,7 @@ public Sequence runSimple( final Sequence results; try { - final Access access = authorize(authenticationResult, null); + final Access access = authorize(authenticationResult); if (!access.isAllowed()) { throw new ISE("Unauthorized"); } @@ -182,6 +183,39 @@ public void initialize(final Query baseQuery) this.toolChest = warehouse.getToolChest(baseQuery); } + /** + * Authorize the query. Will return an Access object denoting whether the query is authorized or not. + * + * @param token authentication token from the request + * @param namespace namespace of the authentication token + * @param authenticationResult authentication result of the request + * + * @return authorization result + * + * */ + public Access authorize( + @Nullable final AuthenticationResult authenticationResult + ) + { + transition(State.INITIALIZED, State.AUTHORIZING); + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + authenticationResult, + Iterables.transform( + queryPlus.getQuery().getDataSource().getNames(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR + ), + authorizerMapper + ); + + if (!authResult.isAllowed()) { + // Not authorized; go straight to Jail, do not pass Go. + transition(State.AUTHORIZING, State.DONE); + } else { + transition(State.AUTHORIZING, State.AUTHORIZED); + } + return authResult; + } + /** * Authorize the query. Will return an Access object denoting whether the query is authorized or not. * @@ -194,27 +228,18 @@ public void initialize(final Query baseQuery) * * */ public Access authorize( - @Nullable final AuthenticationResult authenticationResult, @Nullable HttpServletRequest req ) { transition(State.INITIALIZED, State.AUTHORIZING); - Access authResult; - if (req != null) { - authResult = AuthorizationUtils.authorizeAllResourceActions( - req, - queryPlus.getQuery().getDataSource().getNames(), - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authorizerMapper - ); - } else { - authResult = AuthorizationUtils.authorizeAllResourceActions( - queryPlus.getQuery().getDataSource().getNames(), - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, - authenticationResult, - authorizerMapper - ); - } + Access authResult = AuthorizationUtils.authorizeAllResourceActions( + req, + Iterables.transform( + queryPlus.getQuery().getDataSource().getNames(), + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR + ), + authorizerMapper + ); if (!authResult.isAllowed()) { // Not authorized; go straight to Jail, do not pass Go. diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 2879c8ae0085..2cda222e4655 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import com.google.common.io.CountingOutputStream; import com.google.inject.Inject; @@ -45,9 +46,9 @@ import io.druid.server.metrics.QueryCountStatsProvider; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; -import io.druid.server.security.AuthenticationResult; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import org.joda.time.DateTime; import javax.servlet.http.HttpServletRequest; @@ -137,13 +138,12 @@ public Response getServer(@PathParam("id") String queryId, @Context final HttpSe Access authResult = AuthorizationUtils.authorizeAllResourceActions( req, - datasources, - AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR, + Iterables.transform(datasources, AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR), authorizerMapper ); if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); + throw new ForbiddenException(authResult.toString()); } queryManager.cancelQuery(queryId); @@ -176,12 +176,9 @@ public Response doPost( log.debug("Got query [%s]", query); } - final Access authResult = queryLifecycle.authorize( - (AuthenticationResult) req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT), - req - ); + final Access authResult = queryLifecycle.authorize(req); if (!authResult.isAllowed()) { - return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build(); + throw new ForbiddenException(authResult.toString()); } final QueryLifecycle.QueryResponse queryResponse = queryLifecycle.execute(); @@ -275,6 +272,12 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1); return context.gotError(e); } + catch (ForbiddenException e) { + // don't do anything for an authorization failure, ForbiddenExceptionMapper will catch this later and + // send an error response if this is thrown. + Throwables.propagate(e); + return context.gotError(e); + } catch (Exception e) { failedQueryCount.incrementAndGet(); queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1); diff --git a/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java b/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java index 09e2a955c0b6..406917fd5795 100644 --- a/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/AbstractResourceFilter.java @@ -25,7 +25,6 @@ import com.sun.jersey.spi.container.ContainerResponseFilter; import com.sun.jersey.spi.container.ResourceFilter; import io.druid.server.security.Action; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import javax.servlet.http.HttpServletRequest; @@ -37,16 +36,13 @@ public abstract class AbstractResourceFilter implements ResourceFilter, Containe @Context private HttpServletRequest req; - private final AuthConfig authConfig; private AuthorizerMapper authorizerMapper; @Inject public AbstractResourceFilter( - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; } @@ -67,11 +63,6 @@ public HttpServletRequest getReq() return req; } - public AuthConfig getAuthConfig() - { - return authConfig; - } - public AuthorizerMapper getAuthorizerMapper() { return authorizerMapper; diff --git a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java index 078b4ab5072b..da50a0b0f5be 100644 --- a/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/ConfigResourceFilter.java @@ -21,18 +21,14 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; - /** * Use this ResourceFilter at end points where Druid Cluster configuration is read or written * Here are some example paths where this filter is used - @@ -46,11 +42,10 @@ public class ConfigResourceFilter extends AbstractResourceFilter { @Inject public ConfigResourceFilter( - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizerMapper); + super(authorizerMapper); } @Override @@ -68,11 +63,7 @@ public ContainerRequest filter(ContainerRequest request) ); if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); + throw new ForbiddenException(authResult.toString()); } return request; diff --git a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java index c3e85c7ad65e..9a489e6058a4 100644 --- a/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/DatasourceResourceFilter.java @@ -25,18 +25,15 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; -import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.PathSegment; -import javax.ws.rs.core.Response; import java.util.List; /** @@ -50,11 +47,10 @@ public class DatasourceResourceFilter extends AbstractResourceFilter { @Inject public DatasourceResourceFilter( - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizerMapper); + super(authorizerMapper); } @Override @@ -72,11 +68,7 @@ public ContainerRequest filter(ContainerRequest request) ); if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); + throw new ForbiddenException(authResult.toString()); } return request; diff --git a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java index a5a3c13bcafc..7a6223c3063e 100644 --- a/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/RulesResourceFilter.java @@ -25,18 +25,15 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; -import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.PathSegment; -import javax.ws.rs.core.Response; import java.util.List; @@ -50,11 +47,10 @@ public class RulesResourceFilter extends AbstractResourceFilter { @Inject public RulesResourceFilter( - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizerMapper); + super(authorizerMapper); } @Override @@ -88,11 +84,7 @@ public boolean apply(PathSegment input) ); if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); + throw new ForbiddenException(authResult.toString()); } return request; diff --git a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java index db7a99485286..f3f8a16343a7 100644 --- a/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/io/druid/server/http/security/StateResourceFilter.java @@ -21,18 +21,14 @@ import com.google.inject.Inject; import com.sun.jersey.spi.container.ContainerRequest; -import io.druid.java.util.common.StringUtils; import io.druid.server.security.Access; -import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; import io.druid.server.security.ResourceType; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; - /** * Use this ResourceFilter at end points where Druid Cluster State is read or written * Here are some example paths where this filter is used - @@ -52,11 +48,10 @@ public class StateResourceFilter extends AbstractResourceFilter { @Inject public StateResourceFilter( - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { - super(authConfig, authorizerMapper); + super(authorizerMapper); } @Override @@ -74,11 +69,7 @@ public ContainerRequest filter(ContainerRequest request) ); if (!authResult.isAllowed()) { - throw new WebApplicationException( - Response.status(Response.Status.FORBIDDEN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build() - ); + throw new ForbiddenException(authResult.toString()); } return request; diff --git a/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java index c66da3e8a0d2..56b7b0097cd1 100644 --- a/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java +++ b/server/src/main/java/io/druid/server/initialization/AuthenticatorMapperModule.java @@ -31,6 +31,7 @@ import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.initialization.DruidModule; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; @@ -90,11 +91,14 @@ public AuthenticatorMapper get() List authenticators = authConfig.getAuthenticatorChain(); - // If user didn't configure any Authenticators, use the default which accepts all requests. - if (authenticators == null || authenticators.isEmpty()) { - Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); - return new AuthenticatorMapper(defaultMap, "allowAll"); + // Default configuration is to allow all requests. + if (authenticators == null) { + authenticatorMap.put("allowAll", new AllowAllAuthenticator()); + return new AuthenticatorMapper(authenticatorMap, "allowAll"); + } + + if (authenticators.isEmpty()) { + throw new IAE("Must have at least one Authenticator configured."); } for (String authenticatorName : authenticators) { diff --git a/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java b/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java index 86c975c60565..54e5eda36a70 100644 --- a/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java +++ b/server/src/main/java/io/druid/server/initialization/AuthorizerMapperModule.java @@ -31,6 +31,7 @@ import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.initialization.DruidModule; +import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; @@ -86,11 +87,10 @@ public void inject(Injector injector, Properties props, JsonConfigurator configu public AuthorizerMapper get() { Map authorizerMap = Maps.newHashMap(); - List authorizers = authConfig.getAuthorizers(); - // If user didn't configure any Authorizers, use the default which accepts all requests. - if (authorizers == null || authorizers.isEmpty()) { + // Default is allow all + if (authorizers == null) { return new AuthorizerMapper(null) { @Override public Authorizer getAuthorizer(String name) @@ -100,6 +100,10 @@ public Authorizer getAuthorizer(String name) }; } + if (authorizers.isEmpty()) { + throw new IAE("Must have at least one Authorizer configured."); + } + for (String authorizerName : authorizers) { final String authorizerPropertyBase = StringUtils.format(AUTHORIZER_PROPERTIES_FORMAT_STRING, authorizerName); final JsonConfigProvider authorizerProvider = new JsonConfigProvider<>( diff --git a/server/src/main/java/io/druid/server/initialization/jetty/ForbiddenExceptionMapper.java b/server/src/main/java/io/druid/server/initialization/jetty/ForbiddenExceptionMapper.java new file mode 100644 index 000000000000..6c12a4fd2633 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/jetty/ForbiddenExceptionMapper.java @@ -0,0 +1,44 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.initialization.jetty; + + +import com.google.common.collect.ImmutableMap; +import io.druid.server.security.ForbiddenException; + +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.ext.ExceptionMapper; +import javax.ws.rs.ext.Provider; + +@Provider +public class ForbiddenExceptionMapper implements ExceptionMapper +{ + @Override + public Response toResponse(ForbiddenException exception) + { + return Response.status(Response.Status.FORBIDDEN) + .type(MediaType.APPLICATION_JSON) + .entity(ImmutableMap.of( + "Access-Check-Result", exception.getMessage() + )) + .build(); + } +} diff --git a/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java b/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java index a7c17757ed68..278d752a478a 100644 --- a/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java +++ b/server/src/main/java/io/druid/server/initialization/jetty/JettyServerModule.java @@ -100,6 +100,7 @@ protected void configureServlets() binder.bind(GuiceContainer.class).to(DruidGuiceContainer.class); binder.bind(DruidGuiceContainer.class).in(Scopes.SINGLETON); binder.bind(CustomExceptionMapper.class).in(Singleton.class); + binder.bind(ForbiddenExceptionMapper.class).in(Singleton.class); serve("/*").with(DruidGuiceContainer.class); diff --git a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java index d1856bf92f1d..e73c8bd73ae9 100644 --- a/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java +++ b/server/src/main/java/io/druid/server/lookup/cache/LookupCoordinatorManager.java @@ -47,7 +47,7 @@ import io.druid.concurrent.Execs; import io.druid.concurrent.LifecycleLock; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.IAE; import io.druid.java.util.common.IOE; @@ -57,7 +57,6 @@ import io.druid.query.lookup.LookupsState; import io.druid.server.http.HostAndPortWithScheme; import io.druid.server.listener.resource.ListenerResource; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -134,20 +133,19 @@ public class LookupCoordinatorManager @Inject public LookupCoordinatorManager( - final @Global HttpClient httpClient, + final @EscalatedGlobal HttpClient httpClient, final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, final @Smile ObjectMapper smileMapper, final JacksonConfigManager configManager, - final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig, - final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper - ) + final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig + ) { this( druidNodeDiscoveryProvider, configManager, lookupCoordinatorManagerConfig, new LookupsCommunicator( - authenticatorHttpClientWrapper.getEscalatedClient(httpClient), + httpClient, lookupCoordinatorManagerConfig, smileMapper ), diff --git a/server/src/main/java/io/druid/server/security/AuthConfig.java b/server/src/main/java/io/druid/server/security/AuthConfig.java index 79be88406b16..e93d84939706 100644 --- a/server/src/main/java/io/druid/server/security/AuthConfig.java +++ b/server/src/main/java/io/druid/server/security/AuthConfig.java @@ -38,26 +38,21 @@ public class AuthConfig public AuthConfig() { - this(false, null, null, null); + this(null, null, null); } @JsonCreator public AuthConfig( - @JsonProperty("enabled") boolean enabled, @JsonProperty("authenticatorChain") List authenticationChain, @JsonProperty("escalatedAuthenticator") String escalatedAuthenticator, @JsonProperty("authorizers") List authorizers ) { - this.enabled = enabled; this.authenticatorChain = authenticationChain; this.escalatedAuthenticator = escalatedAuthenticator == null ? "allowAll" : escalatedAuthenticator; this.authorizers = authorizers; } - @JsonProperty - private final boolean enabled; - @JsonProperty private final List authenticatorChain; @@ -67,11 +62,6 @@ public AuthConfig( @JsonProperty private List authorizers; - public boolean isEnabled() - { - return enabled; - } - public List getAuthenticatorChain() { return authenticatorChain; @@ -91,8 +81,7 @@ public List getAuthorizers() public String toString() { return "AuthConfig{" + - "enabled=" + enabled + - ", authenticatorChain='" + authenticatorChain + '\'' + + "authenticatorChain='" + authenticatorChain + '\'' + ", escalatedAuthenticator='" + escalatedAuthenticator + '\'' + ", authorizers='" + authorizers + '\'' + '}'; @@ -110,9 +99,6 @@ public boolean equals(Object o) AuthConfig that = (AuthConfig) o; - if (isEnabled() != that.isEnabled()) { - return false; - } if (getAuthenticatorChain() != null ? !getAuthenticatorChain().equals(that.getAuthenticatorChain()) : that.getAuthenticatorChain() != null) { @@ -123,20 +109,16 @@ public boolean equals(Object o) : that.getEscalatedAuthenticator() != null) { return false; } - return getAuthorizers() != null - ? getAuthorizers().equals(that.getAuthorizers()) - : that.getAuthorizers() == null; + return getAuthorizers() != null ? getAuthorizers().equals(that.getAuthorizers()) : that.getAuthorizers() == null; } @Override public int hashCode() { - int result = (isEnabled() ? 1 : 0); - result = 31 * result + (getAuthenticatorChain() != null ? getAuthenticatorChain().hashCode() : 0); + int result = getAuthenticatorChain() != null ? getAuthenticatorChain().hashCode() : 0; result = 31 * result + (getEscalatedAuthenticator() != null ? getEscalatedAuthenticator().hashCode() : 0); result = 31 * result + (getAuthorizers() != null ? getAuthorizers().hashCode() : 0); return result; } - } diff --git a/server/src/main/java/io/druid/server/security/AuthTestUtils.java b/server/src/main/java/io/druid/server/security/AuthTestUtils.java new file mode 100644 index 000000000000..e06fa9c23030 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthTestUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import com.google.common.collect.Maps; + +import java.util.Map; + +public class AuthTestUtils +{ + public static final AuthenticatorMapper TEST_AUTHENTICATOR_MAPPER; + public static final AuthorizerMapper TEST_AUTHORIZER_MAPPER; + + static { + final Map defaultMap = Maps.newHashMap(); + defaultMap.put("allowAll", new AllowAllAuthenticator()); + TEST_AUTHENTICATOR_MAPPER = new AuthenticatorMapper(defaultMap, "allowAll"); + + TEST_AUTHORIZER_MAPPER = new AuthorizerMapper(null) { + @Override + public Authorizer getAuthorizer(String name) + { + return new AllowAllAuthorizer(); + } + }; + } +} diff --git a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java index d83dc1d2ccd3..62697885ef0c 100644 --- a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java @@ -69,13 +69,12 @@ public static void addSecuritySanityCheckFilter( public static void addPreResponseAuthorizationCheckFilter( ServletContextHandler root, List authenticators, - ObjectMapper jsonMapper, - AuthConfig authConfig + ObjectMapper jsonMapper ) { root.addFilter( new FilterHolder( - new PreResponseAuthorizationCheckFilter(authConfig, authenticators, jsonMapper) + new PreResponseAuthorizationCheckFilter(authenticators, jsonMapper) ), "/*", null diff --git a/server/src/main/java/io/druid/server/security/Authenticator.java b/server/src/main/java/io/druid/server/security/Authenticator.java index 1fe925e117b2..50468cbfd5ea 100644 --- a/server/src/main/java/io/druid/server/security/Authenticator.java +++ b/server/src/main/java/io/druid/server/security/Authenticator.java @@ -24,6 +24,7 @@ import com.metamx.http.client.HttpClient; import io.druid.server.initialization.jetty.ServletFilterHolder; +import javax.annotation.Nullable; import javax.servlet.Filter; import java.util.Map; @@ -72,10 +73,12 @@ public interface Authenticator extends ServletFilterHolder * Return a WWW-Authenticate challenge scheme string appropriate for this Authenticator's authentication mechanism. *

* For example, a Basic HTTP implementation should return "Basic", while a Kerberos implementation would return - * "Negotiate". + * "Negotiate". If this method returns null, no authentication scheme will be added for that Authenticator + * implementation. * * @return Authentication scheme */ + @Nullable public String getAuthChallengeHeader(); /** @@ -88,8 +91,10 @@ public interface Authenticator extends ServletFilterHolder * * @param context JDBC connection context * - * @return true if the identity represented by the context is successfully authenticated + * @return AuthenticationResult of the identity represented by the context is successfully authenticated, + * null if authentication failed */ + @Nullable public AuthenticationResult authenticateJDBCContext(Map context); /** diff --git a/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java b/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java index 59eb74f1dfcd..952f607cbaf8 100644 --- a/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java +++ b/server/src/main/java/io/druid/server/security/AuthenticatorMapper.java @@ -22,7 +22,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import io.druid.guice.ManageLifecycle; -import io.druid.java.util.common.lifecycle.LifecycleStart; import java.util.List; import java.util.Map; @@ -47,11 +46,6 @@ public AuthenticatorMapper( ); } - public Authenticator getAuthenticator(String namespace) - { - return authenticatorMap.get(namespace); - } - public Authenticator getEscalatingAuthenticator() { return escalatingAuthenticator; @@ -61,20 +55,4 @@ public List getAuthenticatorChain() { return Lists.newArrayList(authenticatorMap.values()); } - - @LifecycleStart - public void start() - { - for (Authenticator authenticator : authenticatorMap.values()) { - //authenticator.start(); - } - } - - @LifecycleStart - public void stop() - { - for (Authenticator authenticator : authenticatorMap.values()) { - //authenticator.stop(); - } - } } diff --git a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java index 6f2d0c5e2616..5c8fb2e41361 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthorizationUtils.java @@ -65,33 +65,23 @@ public static Access authorizeResourceAction( } /** - * Check a list of resource-actions using the authorization fields from the request. + * Check a list of resource-actions to be performed by the identity represented by authenticationResult. * * If one of the resource-actions fails the authorization check, this method returns the failed * Access object from the check. * * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. * - * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. - * - * If this attribute is already set when this function is called, an exception is thrown. - * @param request HTTP request to be authorized - * @param resourceActions A list of resource-actions to authorize + * @param authenticationResult Authentication result representing identity of requester + * @param resourceActions An Iterable of resource-actions to authorize * @return ACCESS_OK or the Access object from the first failed check */ public static Access authorizeAllResourceActions( - final HttpServletRequest request, - final List resourceActions, + final AuthenticationResult authenticationResult, + final Iterable resourceActions, final AuthorizerMapper authorizerMapper ) { - final AuthenticationResult authenticationResult = (AuthenticationResult) request.getAttribute( - AuthConfig.DRUID_AUTHENTICATION_RESULT - ); - if (authenticationResult == null) { - throw new ISE("Null authentication result"); - } - final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); if (authorizer == null) { throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); @@ -110,20 +100,17 @@ public static Access authorizeAllResourceActions( resourceAction.getAction() ); if (!access.isAllowed()) { - request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); return access; } else { resultCache.add(resourceAction); } } - request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); return Access.OK; } /** - * Check a list of caller-defined resources, after converting them into a list of resource-actions - * using a caller provided function. + * Check a list of resource-actions to be performed as a result of an HTTP request. * * If one of the resource-actions fails the authorization check, this method returns the failed * Access object from the check. @@ -132,21 +119,22 @@ public static Access authorizeAllResourceActions( * * This function will set the DRUID_AUTHORIZATION_CHECKED attribute in the request. * - * If this attribute is already set when this fImmutableList.>of(SupervisorManager.class, Authorizer.class)unction is called, an exception is thrown. + * If this attribute is already set when this function is called, an exception is thrown. * - * @param request HTTP request to be generated - * @param resources List of resources - * @param raGenerator Function that creates a resource-action from a resource - * @param Type of the resources in the resource list + * @param request HTTP request to be authorized + * @param resourceActions An Iterable of resource-actions to authorize * @return ACCESS_OK or the Access object from the first failed check */ - public static Access authorizeAllResourceActions( + public static Access authorizeAllResourceActions( final HttpServletRequest request, - final Collection resources, - final Function raGenerator, + final Iterable resourceActions, final AuthorizerMapper authorizerMapper ) { + if (request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED) != null) { + throw new ISE("Request already had authorization check."); + } + final AuthenticationResult authenticationResult = (AuthenticationResult) request.getAttribute( AuthConfig.DRUID_AUTHENTICATION_RESULT ); @@ -154,87 +142,14 @@ public static Access authorizeAllResourceActions( throw new ISE("Null authentication result"); } - final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); - if (authorizer == null) { - throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); - } - - // this method returns on first failure, so only successful Access results are kept in the cache - final Set resultCache = Sets.newHashSet(); - - for (ResType resource : resources) { - final ResourceAction resourceAction = raGenerator.apply(resource); - if (resultCache.contains(resourceAction)) { - continue; - } - final Access access = authorizer.authorize( - authenticationResult, - resourceAction.getResource(), - resourceAction.getAction() - ); - if (!access.isAllowed()) { - request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, false); - return access; - } else { - resultCache.add(resourceAction); - } - } - - request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - return Access.OK; - } - - /** - * Check a list of caller-defined resources, after converting them into a list of resource-actions - * using a caller provided function. - * - * If one of the resource-actions fails the authorization check, this method returns the failed - * Access object from the check. - * - * Otherwise, return ACCESS_OK if all resource-actions were successfully authorized. - * - * @param resources List of resources - * @param raGenerator Function that creates a resource-action from a resource - * @param Type of the resources in the resource list - * @return ACCESS_OK or the Access object from the first failed check - */ - public static Access authorizeAllResourceActions( - final Collection resources, - final Function raGenerator, - final AuthenticationResult authenticationResult, - final AuthorizerMapper authorizerMapper - ) - { - if (authenticationResult == null) { - throw new ISE("null authentication result"); - } - - final Authorizer authorizer = authorizerMapper.getAuthorizer(authenticationResult.getAuthorizerName()); - if (authorizer == null) { - throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); - } - - // this method returns on first failure, so only successful Access results are kept in the cache - final Set resultCache = Sets.newHashSet(); - - for (ResType resource : resources) { - final ResourceAction resourceAction = raGenerator.apply(resource); - if (resultCache.contains(resourceAction)) { - continue; - } - final Access access = authorizer.authorize( - authenticationResult, - resourceAction.getResource(), - resourceAction.getAction() - ); - if (!access.isAllowed()) { - return access; - } else { - resultCache.add(resourceAction); - } - } + Access access = authorizeAllResourceActions( + authenticationResult, + resourceActions, + authorizerMapper + ); - return Access.OK; + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, access.isAllowed()); + return access; } /** @@ -269,26 +184,26 @@ public static List filterAuthorizedResources( throw new ISE("No authorizer found with name: [%s].", authenticationResult.getAuthorizerName()); } - int initialSize = resources.size(); final Map resultCache = Maps.newHashMap(); List filteredResources = new ArrayList<>(); for (ResType resource : resources) { final ResourceAction resourceAction = resourceActionGenerator.apply(resource); - Access access = resultCache.get(resourceAction); - if (access == null) { - access = authorizer.authorize( - authenticationResult, - resourceAction.getResource(), - resourceAction.getAction() - ); - resultCache.put(resourceAction, access); - } + Access access = resultCache.computeIfAbsent( + resourceAction, + ra -> authorizer.authorize( + authenticationResult, + ra.getResource(), + ra.getAction() + ) + ); if (access.isAllowed()) { filteredResources.add(resource); } } - request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, (filteredResources.size() > 0 || initialSize == 0)); + // We're filtering, so having access to none of the objects isn't an authorization failure (in terms of whether + // to send an error response or not.) + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); return filteredResources; } diff --git a/server/src/main/java/io/druid/server/security/Authorizer.java b/server/src/main/java/io/druid/server/security/Authorizer.java index 3580bdb51857..9cd586c9f9dc 100644 --- a/server/src/main/java/io/druid/server/security/Authorizer.java +++ b/server/src/main/java/io/druid/server/security/Authorizer.java @@ -24,7 +24,6 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "denyAll", value = DenyAllAuthorizer.class), @JsonSubTypes.Type(name = "allowAll", value = AllowAllAuthorizer.class) }) /** @@ -47,7 +46,7 @@ public interface Authorizer * @param resource The resource to be accessed * @param action The action to perform on the resource * - * @return An Access object representing the result of the authorization check. + * @return An Access object representing the result of the authorization check. Must not be null. */ Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action); } diff --git a/server/src/main/java/io/druid/server/security/AuthorizerMapper.java b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java index 26487bf2697f..2c029aafe038 100644 --- a/server/src/main/java/io/druid/server/security/AuthorizerMapper.java +++ b/server/src/main/java/io/druid/server/security/AuthorizerMapper.java @@ -20,7 +20,6 @@ package io.druid.server.security; import io.druid.guice.ManageLifecycle; -import io.druid.java.util.common.lifecycle.LifecycleStart; import java.util.Map; @@ -40,14 +39,4 @@ public Authorizer getAuthorizer(String name) { return authorizerMap.get(name); } - - @LifecycleStart - public void start() - { - } - - @LifecycleStart - public void stop() - { - } } diff --git a/server/src/main/java/io/druid/server/security/ForbiddenException.java b/server/src/main/java/io/druid/server/security/ForbiddenException.java new file mode 100644 index 000000000000..a317a8b94447 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/ForbiddenException.java @@ -0,0 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +/** + * Throw this when a request is unauthorized and we want to send a 403 response back, Jersey exception mapper will + * take care of sending the response. + */ +public class ForbiddenException extends SecurityException +{ + public ForbiddenException() + { + super("Unauthorized."); + } + + public ForbiddenException(String msg) + { + super(msg); + } +} diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index 6a5273f69139..77b48a116d6b 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -21,9 +21,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; +import com.metamx.emitter.EmittingLogger; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.logger.Logger; import io.druid.query.QueryInterruptedException; import io.druid.server.DruidNode; import org.eclipse.jetty.server.Response; @@ -49,19 +49,16 @@ */ public class PreResponseAuthorizationCheckFilter implements Filter { - private static final Logger log = new Logger(PreResponseAuthorizationCheckFilter.class); + private static final EmittingLogger log = new EmittingLogger(PreResponseAuthorizationCheckFilter.class); - private final AuthConfig authConfig; private final List authenticators; private final ObjectMapper jsonMapper; public PreResponseAuthorizationCheckFilter( - AuthConfig authConfig, List authenticators, ObjectMapper jsonMapper ) { - this.authConfig = authConfig; this.authenticators = authenticators; this.jsonMapper = jsonMapper; } @@ -117,8 +114,15 @@ public void doFilter( // Note: rather than throwing an exception here, it would be nice to blank out the original response // since the request didn't have any authorization checks performed. However, this breaks proxying // (e.g. OverlordServletProxy), so this is not implemented for now. - log.error(errorMsg); - throw new ISE(errorMsg); + handleAuthorizationCheckError(errorMsg, servletResponse); + } + + if (authInfoChecked != null && !authInfoChecked && response.getStatus() != Response.SC_FORBIDDEN) { + String errorMsg = StringUtils.format( + "Request's authorization check failed but status code was not 403: %s", + ((HttpServletRequest) servletRequest).getRequestURI() + ); + handleAuthorizationCheckError(errorMsg, servletResponse); } } @@ -128,6 +132,25 @@ public void destroy() } + private void handleAuthorizationCheckError(String errorMsg, ServletResponse servletResponse) + { + log.error(errorMsg); + + // Send out an alert so there's a centralized collection point for seeing errors of this nature + log.makeAlert(errorMsg); + + if (servletResponse.isCommitted()) { + throw new ISE(errorMsg); + } else { + try { + ((HttpServletResponse) servletResponse).sendError(Response.SC_FORBIDDEN); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + } + private static boolean errorOverridesMissingAuth(int status) { return status == Response.SC_INTERNAL_SERVER_ERROR; diff --git a/server/src/main/java/io/druid/server/security/Resource.java b/server/src/main/java/io/druid/server/security/Resource.java index e2b1e86292b7..ca0a64a12af3 100644 --- a/server/src/main/java/io/druid/server/security/Resource.java +++ b/server/src/main/java/io/druid/server/security/Resource.java @@ -75,4 +75,13 @@ public int hashCode() result = 31 * result + type.hashCode(); return result; } + + @Override + public String toString() + { + return "Resource{" + + "name='" + name + '\'' + + ", type=" + type + + '}'; + } } diff --git a/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java b/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java index 9980c97c5fdd..1fbbc7013c83 100644 --- a/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java +++ b/server/src/main/java/io/druid/server/security/UnsecuredResourceFilter.java @@ -29,10 +29,6 @@ public class UnsecuredResourceFilter implements Filter { - public UnsecuredResourceFilter() - { - } - @Override public void init(FilterConfig filterConfig) throws ServletException { diff --git a/server/src/test/java/io/druid/client/BrokerServerViewTest.java b/server/src/test/java/io/druid/client/BrokerServerViewTest.java index 994779e85392..2dae5d4310a9 100644 --- a/server/src/test/java/io/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/io/druid/client/BrokerServerViewTest.java @@ -43,8 +43,6 @@ import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; -import io.druid.server.security.AuthenticatorHttpClientWrapper; -import io.druid.server.security.AllowAllAuthenticator; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -339,7 +337,6 @@ public CallbackAction segmentViewInitialized() EasyMock.createMock(QueryWatcher.class), getSmileMapper(), EasyMock.createMock(HttpClient.class), - new AuthenticatorHttpClientWrapper(new AllowAllAuthenticator()), baseView, new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), new NoopServiceEmitter(), diff --git a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java index 65c42f3bc6f4..09a46e19c3b1 100644 --- a/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/HttpServerInventoryViewTest.java @@ -174,8 +174,7 @@ DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerT httpClient, druidNodeDiscoveryProvider, Predicates.alwaysTrue(), - new HttpServerInventoryViewConfig(null, null, null), - null + new HttpServerInventoryViewConfig(null, null, null) ); CountDownLatch initializeCallback1 = new CountDownLatch(1); diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index 0152a183d203..8e17dcce08c5 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -32,10 +32,8 @@ import io.druid.server.metrics.EventReceiverFirehoseMetric; import io.druid.server.metrics.EventReceiverFirehoseRegister; import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import org.apache.commons.io.IOUtils; import org.easymock.EasyMock; import org.junit.Assert; @@ -68,14 +66,6 @@ public class EventReceiverFirehoseTest private EventReceiverFirehoseFactory.EventReceiverFirehose firehose; private EventReceiverFirehoseRegister register = new EventReceiverFirehoseRegister(); private HttpServletRequest req; - private AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) - { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }; @Before public void setUp() throws Exception @@ -88,7 +78,7 @@ public void setUp() throws Exception new DefaultObjectMapper(), new DefaultObjectMapper(), register, - authorizerMapper + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); firehose = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory.connect( new MapInputRowParser( @@ -109,6 +99,9 @@ public void setUp() throws Exception @Test public void testSingleThread() throws IOException { + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) .anyTimes(); @@ -156,6 +149,9 @@ public void testSingleThread() throws IOException @Test public void testMultipleThreads() throws InterruptedException, IOException, TimeoutException, ExecutionException { + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) .anyTimes(); @@ -231,7 +227,7 @@ public void testDuplicateRegistering() throws IOException new DefaultObjectMapper(), new DefaultObjectMapper(), register, - authorizerMapper + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); EventReceiverFirehoseFactory.EventReceiverFirehose firehose2 = (EventReceiverFirehoseFactory.EventReceiverFirehose) eventReceiverFirehoseFactory2 @@ -254,6 +250,9 @@ public void testDuplicateRegistering() throws IOException @Test(timeout = 40_000L) public void testShutdownWithPrevTime() throws Exception { + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) .anyTimes(); @@ -270,6 +269,9 @@ public void testShutdownWithPrevTime() throws Exception @Test(timeout = 40_000L) public void testShutdown() throws Exception { + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) .anyTimes(); diff --git a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java index a0130c238710..340503bcadd1 100644 --- a/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java +++ b/server/src/test/java/io/druid/server/AsyncQueryForwardingServletTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; import com.google.common.net.HostAndPort; import com.google.inject.Binder; import com.google.inject.Inject; @@ -52,12 +51,10 @@ import io.druid.server.log.RequestLogger; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.server.router.QueryHostFinder; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.Authorizer; import io.druid.server.security.AllowAllAuthorizer; +import io.druid.server.security.AuthTestUtils; +import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthorizerMapper; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; @@ -79,7 +76,6 @@ import java.net.URI; import java.net.URL; import java.util.Collection; -import java.util.Map; import java.util.concurrent.CountDownLatch; public class AsyncQueryForwardingServletTest extends BaseJettyTest @@ -238,8 +234,6 @@ public Collection getAllServers() } }; - Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); ObjectMapper jsonMapper = injector.getInstance(ObjectMapper.class); ServletHolder holder = new ServletHolder( new AsyncQueryForwardingServlet( @@ -259,7 +253,7 @@ public void log(RequestLogLine requestLogLine) throws IOException } }, new DefaultGenericQueryMetricsFactory(jsonMapper), - new AuthenticatorMapper(defaultMap, "allowAll") + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER ) { @Override diff --git a/server/src/test/java/io/druid/server/QueryResourceTest.java b/server/src/test/java/io/druid/server/QueryResourceTest.java index 46898437ad15..879672b5dc19 100644 --- a/server/src/test/java/io/druid/server/QueryResourceTest.java +++ b/server/src/test/java/io/druid/server/QueryResourceTest.java @@ -45,9 +45,11 @@ import io.druid.server.security.Access; import io.druid.server.security.Action; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthTestUtils; import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -146,7 +148,7 @@ public void setup() new NoopRequestLogger(), serverConfig, new AuthConfig(), - new AuthorizerMapper(null) + AuthTestUtils.TEST_AUTHORIZER_MAPPER ), jsonMapper, jsonMapper, @@ -175,10 +177,17 @@ public void setup() @Test public void testGoodQuery() throws IOException { + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(authenticationResult) .anyTimes(); + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + EasyMock.replay(testServletRequest); Response response = queryResource.doPost( new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")), @@ -204,6 +213,10 @@ public void testBadQuery() throws IOException @Test public void testSecuredQuery() throws Exception { + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(authenticationResult) .anyTimes(); @@ -244,25 +257,30 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper ), jsonMapper, jsonMapper, queryManager, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper, new DefaultGenericQueryMetricsFactory(jsonMapper) ); - Response response = queryResource.doPost( - new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")), - null /*pretty*/, - testServletRequest - ); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), response.getStatus()); - response = queryResource.doPost( + try { + queryResource.doPost( + new ByteArrayInputStream(simpleTimeSeriesQuery.getBytes("UTF-8")), + null /*pretty*/, + testServletRequest + ); + Assert.fail("doPost did not throw ForbiddenException for an unauthorized query"); + } + catch (ForbiddenException e) { + } + + Response response = queryResource.doPost( new ByteArrayInputStream("{\"queryType\":\"timeBoundary\", \"dataSource\":\"allow\"}".getBytes("UTF-8")), null /*pretty*/, testServletRequest @@ -280,6 +298,10 @@ public void testSecuredGetServer() throws Exception final CountDownLatch startAwaitLatch = new CountDownLatch(1); final CountDownLatch cancelledCountDownLatch = new CountDownLatch(1); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(authenticationResult) .anyTimes(); @@ -332,13 +354,13 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper ), jsonMapper, jsonMapper, queryManager, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper, new DefaultGenericQueryMetricsFactory(jsonMapper) ); @@ -400,6 +422,10 @@ public void testDenySecuredGetServer() throws Exception final CountDownLatch waitFinishLatch = new CountDownLatch(2); final CountDownLatch startAwaitLatch = new CountDownLatch(1); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(authenticationResult) .anyTimes(); @@ -450,13 +476,13 @@ public Access authorize(AuthenticationResult authenticationResult, Resource reso new NoopServiceEmitter(), new NoopRequestLogger(), serverConfig, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper ), jsonMapper, jsonMapper, queryManager, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper, new DefaultGenericQueryMetricsFactory(jsonMapper) ); @@ -500,10 +526,13 @@ public void run() @Override public void run() { - Response response = queryResource.getServer("id_1", testServletRequest); - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), response.getStatus()); - waitForCancellationLatch.countDown(); - waitFinishLatch.countDown(); + try { + queryResource.getServer("id_1", testServletRequest); + } + catch (ForbiddenException e) { + waitForCancellationLatch.countDown(); + waitFinishLatch.countDown(); + } } } ); diff --git a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java index 178f2d463bd2..3b98960b8e13 100644 --- a/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java +++ b/server/src/test/java/io/druid/server/http/DatasourcesResourceTest.java @@ -29,8 +29,8 @@ import io.druid.server.coordination.ServerType; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthTestUtils; import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authorizer; import io.druid.server.security.AuthorizerMapper; @@ -54,15 +54,6 @@ public class DatasourcesResourceTest { - private AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { - - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }; - private CoordinatorServerView inventoryView; private DruidServer server; private List listDataSources; @@ -142,7 +133,7 @@ public void testGetFullQueryableDataSources() throws Exception null, null, new AuthConfig(), - authorizerMapper + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); Response response = datasourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); @@ -202,7 +193,7 @@ public Access authorize(AuthenticationResult authenticationResult1, Resource res inventoryView, null, null, - new AuthConfig(true, null, null, null), + new AuthConfig(null, null, null), authMapper ); Response response = datasourcesResource.getQueryableDataSources("full", null, request); @@ -251,7 +242,7 @@ public void testGetSimpleQueryableDataSources() throws Exception null, null, new AuthConfig(), - authorizerMapper + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); Response response = datasourcesResource.getQueryableDataSources(null, "simple", request); Assert.assertEquals(200, response.getStatus()); diff --git a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java index fe06602969f7..c6d8eff10be6 100644 --- a/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java +++ b/server/src/test/java/io/druid/server/http/IntervalsResourceTest.java @@ -24,11 +24,9 @@ import io.druid.client.InventoryView; import io.druid.java.util.common.Intervals; import io.druid.server.coordination.ServerType; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; +import io.druid.server.security.AuthTestUtils; import io.druid.server.security.AuthenticationResult; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.joda.time.Interval; @@ -46,14 +44,6 @@ public class IntervalsResourceTest { - private AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }; - private InventoryView inventoryView; private DruidServer server; private List dataSegmentList; @@ -126,7 +116,11 @@ public void testGetIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); expectedIntervals.add(Intervals.of("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); + IntervalsResource intervalsResource = new IntervalsResource( + inventoryView, + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER + ); Response response = intervalsResource.getIntervals(request); TreeMap>> actualIntervals = (TreeMap) response.getEntity(); @@ -155,7 +149,11 @@ public void testSimpleGetSpecificIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); + IntervalsResource intervalsResource = new IntervalsResource( + inventoryView, + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER + ); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", "simple", null, request); Map> actualIntervals = (Map) response.getEntity(); @@ -179,7 +177,11 @@ public void testFullGetSpecificIntervals() List expectedIntervals = new ArrayList<>(); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); + IntervalsResource intervalsResource = new IntervalsResource( + inventoryView, + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER + ); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, "full", request); TreeMap>> actualIntervals = (TreeMap) response.getEntity(); @@ -203,7 +205,11 @@ public void testGetSpecificIntervals() ).atLeastOnce(); EasyMock.replay(inventoryView, request); - IntervalsResource intervalsResource = new IntervalsResource(inventoryView, new AuthConfig(), authorizerMapper); + IntervalsResource intervalsResource = new IntervalsResource( + inventoryView, + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER + ); Response response = intervalsResource.getSpecificIntervals("2010-01-01T00:00:00.000Z/P1D", null, null, request); Map actualIntervals = (Map) response.getEntity(); diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index dec4220cff16..d71bd236f130 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -22,10 +22,10 @@ import com.google.common.collect.Lists; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; +import io.druid.server.security.AllowAllAuthenticator; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authenticator; -import io.druid.server.security.AllowAllAuthenticator; import io.druid.server.security.PreResponseAuthorizationCheckFilter; import org.easymock.EasyMock; import org.junit.Rule; @@ -41,7 +41,6 @@ public class PreResponseAuthorizationCheckFilterTest { private static List authenticators = Lists.newArrayList(new AllowAllAuthenticator()); - private static AuthConfig authConfig = new AuthConfig(true, null, null, null); @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -61,7 +60,6 @@ public void testValidRequest() throws Exception EasyMock.replay(req, resp, filterChain, outputStream); PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( - authConfig, authenticators, new DefaultObjectMapper() ); @@ -88,7 +86,6 @@ public void testAuthenticationFailedRequest() throws Exception EasyMock.replay(req, resp, filterChain, outputStream); PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( - authConfig, authenticators, new DefaultObjectMapper() ); @@ -122,7 +119,6 @@ public void testMissingAuthorizationCheck() throws Exception EasyMock.replay(req, resp, filterChain, outputStream); PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( - authConfig, authenticators, new DefaultObjectMapper() ); diff --git a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java index e6dc9c4466e6..eae5e99cdfed 100644 --- a/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java +++ b/server/src/test/java/io/druid/server/http/security/ResourceFilterTestHelper.java @@ -182,7 +182,7 @@ public void configure(Binder binder) for (Key key : mockableKeys) { binder.bind((Key) key).toInstance(EasyMock.createNiceMock(key.getTypeLiteral().getRawType())); } - binder.bind(AuthConfig.class).toInstance(new AuthConfig(true, null, null, null)); + binder.bind(AuthConfig.class).toInstance(new AuthConfig(null, null, null)); } } ); diff --git a/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java index 7ccd7d9e91f2..92e9ee79b276 100644 --- a/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/SecurityResourceFilterTest.java @@ -37,6 +37,7 @@ import io.druid.server.http.RulesResource; import io.druid.server.http.ServersResource; import io.druid.server.http.TiersResource; +import io.druid.server.security.ForbiddenException; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; @@ -44,8 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; import java.util.Collection; @RunWith(Parameterized.class) @@ -108,7 +107,7 @@ public void testResourcesFilteringAccess() EasyMock.verify(req, request, authorizerMapper); } - @Test(expected = WebApplicationException.class) + @Test(expected = ForbiddenException.class) public void testResourcesFilteringNoAccess() { setUpMockExpectations(requestPath, false, requestMethod); @@ -116,9 +115,9 @@ public void testResourcesFilteringNoAccess() Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath)); try { resourceFilter.getRequestFilter().filter(request); + Assert.fail(); } - catch (WebApplicationException e) { - Assert.assertEquals(Response.Status.FORBIDDEN.getStatusCode(), e.getResponse().getStatus()); + catch (ForbiddenException e) { throw e; } EasyMock.verify(req, request, authorizerMapper); @@ -132,5 +131,4 @@ public void testResourcesFilteringBadPath() Assert.assertFalse(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(badRequestPath)); EasyMock.verify(req, request, authorizerMapper); } - } diff --git a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java index 7e5b65a66387..f9f1219b7bfc 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyQosTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyQosTest.java @@ -43,9 +43,8 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.jetty.JettyBindings; import io.druid.server.initialization.jetty.JettyServerInitializer; -import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthTestUtils; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.AllowAllAuthorizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -78,16 +77,7 @@ public void configure(Binder binder) Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); - binder.bind(AuthorizerMapper.class).toInstance( - new AuthorizerMapper(null) { - - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - } - ); + binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER); JettyBindings.addQosFilter(binder, "/slow/*", 2); final ServerConfig serverConfig = new ObjectMapper().convertValue( ImmutableMap.of("numThreads", "10"), diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 2c401835afc6..1e9206f0b230 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -41,9 +41,8 @@ import io.druid.server.DruidNode; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.initialization.jetty.ServletFilterHolder; -import io.druid.server.security.Authorizer; +import io.druid.server.security.AuthTestUtils; import io.druid.server.security.AuthorizerMapper; -import io.druid.server.security.AllowAllAuthorizer; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Server; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -129,16 +128,7 @@ public EnumSet getDispatcherType() Jerseys.addResource(binder, SlowResource.class); Jerseys.addResource(binder, ExceptionResource.class); Jerseys.addResource(binder, DefaultResource.class); - binder.bind(AuthorizerMapper.class).toInstance( - new AuthorizerMapper(null) { - - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - } - ); + binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER); LifecycleModule.register(binder, Server.class); } } diff --git a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 93805c9ce898..b128b0bc9b5d 100644 --- a/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/io/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -43,8 +43,6 @@ import io.druid.java.util.common.StringUtils; import io.druid.query.lookup.LookupsState; import io.druid.server.http.HostAndPortWithScheme; -import io.druid.server.security.AuthenticatorHttpClientWrapper; -import io.druid.server.security.AllowAllAuthenticator; import org.easymock.EasyMock; import org.joda.time.Duration; import org.junit.After; @@ -79,9 +77,6 @@ public class LookupCoordinatorManagerTest private final HttpClient client = EasyMock.createStrictMock(HttpClient.class); private final JacksonConfigManager configManager = EasyMock.createStrictMock(JacksonConfigManager.class); private final LookupCoordinatorManagerConfig lookupCoordinatorManagerConfig = new LookupCoordinatorManagerConfig(); - private final AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper = new AuthenticatorHttpClientWrapper( - new AllowAllAuthenticator() - ); private static final String LOOKUP_TIER = "lookup_tier"; private static final String SINGLE_LOOKUP_NAME = "lookupName"; @@ -542,8 +537,7 @@ public void testUpdateLookupsFailsUnitialized() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -566,8 +560,7 @@ public void testUpdateLookupsInitialization() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -599,8 +592,7 @@ public void testUpdateLookupAdds() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -637,8 +629,7 @@ public void testUpdateLookupsAddsNewLookup() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -700,8 +691,7 @@ public void testUpdateLookupsOnlyUpdatesToTier() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -756,8 +746,7 @@ public void testUpdateLookupsUpdates() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -787,8 +776,7 @@ public void testUpdateLookupFailsSameVersionUpdates() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -822,8 +810,7 @@ public void testUpdateLookupsAddsNewTier() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -878,8 +865,7 @@ public void testDeleteLookup() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -924,8 +910,7 @@ public void testDeleteLookupIgnoresMissing() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -950,8 +935,7 @@ public void testDeleteLookupIgnoresNotReady() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -977,8 +961,7 @@ public void testGetLookup() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -1007,8 +990,7 @@ public void testGetLookupIgnoresMalformed() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -1033,8 +1015,7 @@ public void testGetLookupIgnoresNotReady() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ) { @Override @@ -1188,8 +1169,7 @@ public void testGetToBeLoadedOnNode() druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ); LookupsState currNodeState = new LookupsState<>( @@ -1225,8 +1205,7 @@ public void testToBeDropped() druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ); LookupsState currNodeState = new LookupsState<>( @@ -1273,8 +1252,7 @@ public void testStartStop() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ); Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); @@ -1311,8 +1289,7 @@ public void testMultipleStartStop() throws Exception druidNodeDiscoveryProvider, mapper, configManager, - lookupCoordinatorManagerConfig, - authenticatorHttpClientWrapper + lookupCoordinatorManagerConfig ); Assert.assertFalse(manager.lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); @@ -1466,8 +1443,7 @@ public long getPeriod() { return 1; } - }, - authenticatorHttpClientWrapper + } ); manager.start(); EasyMock.verify(configManager); diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 3ae124edd497..c13a5424fa6f 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -333,8 +333,7 @@ public void initialize(Server server, Injector injector) AuthenticationUtils.addPreResponseAuthorizationCheckFilter( root, authenticators, - jsonMapper, - authConfig + jsonMapper ); // /status should not redirect, so add first diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index 0500610c15a6..8e06948d4766 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -41,7 +41,7 @@ import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; import io.druid.guice.RouterProcessingModule; -import io.druid.guice.annotations.Global; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Self; import io.druid.guice.http.JettyHttpClientModule; import io.druid.java.util.common.logger.Logger; @@ -57,7 +57,6 @@ import io.druid.server.router.TieredBrokerHostSelector; import io.druid.server.router.TieredBrokerSelectorStrategiesProvider; import io.druid.server.router.TieredBrokerSelectorStrategy; -import io.druid.server.security.AuthenticatorHttpClientWrapper; import org.eclipse.jetty.server.Server; import java.util.List; @@ -139,14 +138,13 @@ public ServerDiscoverySelector getCoordinatorServerDiscoverySelector( @Provides @ManageLifecycle public DruidLeaderClient getLeaderHttpClient( - @Global HttpClient httpClient, + @EscalatedGlobal HttpClient httpClient, DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, - ServerDiscoverySelector serverDiscoverySelector, - AuthenticatorHttpClientWrapper authenticatorHttpClientWrapper + ServerDiscoverySelector serverDiscoverySelector ) { return new DruidLeaderClient( - authenticatorHttpClientWrapper.getEscalatedClient(httpClient), + httpClient, druidNodeDiscoveryProvider, DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, "/druid/coordinator/v1/leader", diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 69fee6876d08..6f3e96cb2e63 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -54,17 +54,15 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer { private static List UNSECURED_PATHS = Lists.newArrayList( - "/", "/favicon.ico", "/css/*", "/druid.js", "/druid.css", "/pages/*", - "/druid/*", "/fonts/*", "/old-console/*", - "/coordinator/*", - "/overlord/*" + "/coordinator/false", + "/overlord/false" ); private static Logger log = new Logger(CoordinatorJettyServerInitializer.class); @@ -126,8 +124,7 @@ public void initialize(Server server, Injector injector) AuthenticationUtils.addPreResponseAuthorizationCheckFilter( root, authenticators, - jsonMapper, - authConfig + jsonMapper ); // /status should not redirect, so add first diff --git a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java index 3ea6afe7e3c2..f66676b9205f 100644 --- a/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/MiddleManagerJettyServerInitializer.java @@ -69,8 +69,7 @@ public void initialize(Server server, Injector injector) AuthenticationUtils.addPreResponseAuthorizationCheckFilter( root, authenticators, - jsonMapper, - authConfig + jsonMapper ); root.addFilter(GuiceFilter.class, "/*", null); diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java index fa3f2e39c4d1..7ebaa2d2f20a 100644 --- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -78,8 +78,7 @@ public void initialize(Server server, Injector injector) AuthenticationUtils.addPreResponseAuthorizationCheckFilter( root, authenticators, - jsonMapper, - authConfig + jsonMapper ); root.addFilter(GuiceFilter.class, "/*", null); diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index a1f1b41c414f..f7acecaa1027 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -97,8 +97,7 @@ public void initialize(Server server, Injector injector) AuthenticationUtils.addPreResponseAuthorizationCheckFilter( root, authenticators, - jsonMapper, - authConfig + jsonMapper ); diff --git a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java index 2f1d1116ffc4..eb1a32961565 100644 --- a/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/io/druid/sql/avatica/DruidMeta.java @@ -38,6 +38,7 @@ import io.druid.server.security.AuthenticationResult; import io.druid.server.security.Authenticator; import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.ForbiddenException; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerFactory; import org.apache.calcite.avatica.MetaImpl; @@ -147,7 +148,7 @@ public StatementHandle prepare( final DruidConnection druidConnection = getDruidConnection(statement.connectionId); AuthenticationResult authenticationResult = authenticateConnection(druidConnection); if (authenticationResult == null) { - throw new SecurityException("Authentication failed."); + throw new ForbiddenException("Authentication failed."); } statement.signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, authenticationResult).getSignature(); return statement; @@ -180,7 +181,7 @@ public ExecuteResult prepareAndExecute( final DruidConnection druidConnection = getDruidConnection(statement.connectionId); AuthenticationResult authenticationResult = authenticateConnection(druidConnection); if (authenticationResult == null) { - throw new SecurityException("Authentication failed."); + throw new ForbiddenException("Authentication failed."); } final Signature signature = druidStatement.prepare(plannerFactory, sql, maxRowCount, authenticationResult).getSignature(); final Frame firstFrame = druidStatement.execute() diff --git a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java index d452d8f8f9d0..d9bf59fba875 100644 --- a/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java +++ b/sql/src/main/java/io/druid/sql/calcite/planner/DruidPlanner.java @@ -23,6 +23,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -32,6 +33,7 @@ import io.druid.server.security.AuthenticatorMapper; import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.ForbiddenException; import io.druid.sql.calcite.rel.DruidConvention; import io.druid.sql.calcite.rel.DruidRel; import org.apache.calcite.DataContext; @@ -99,7 +101,7 @@ public PlannerResult plan( final String sql, final HttpServletRequest request, final AuthenticationResult authenticationResult - ) throws SqlParseException, ValidationException, RelConversionException, SecurityException + ) throws SqlParseException, ValidationException, RelConversionException, ForbiddenException { SqlExplain explain = null; SqlNode parsed = planner.parse(sql); @@ -141,7 +143,7 @@ private PlannerResult planWithDruidConvention( final RelRoot root, final HttpServletRequest request, final AuthenticationResult authenticationResult - ) throws RelConversionException, SecurityException + ) throws RelConversionException, ForbiddenException { final DruidRel druidRel = (DruidRel) planner.transform( Rules.DRUID_CONVENTION_RULES, @@ -158,8 +160,7 @@ private PlannerResult planWithDruidConvention( if (request != null) { authResult = AuthorizationUtils.authorizeAllResourceActions( request, - datasourceNames, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + Iterables.transform(datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR), authorizerMapper ); plannerContext.setAuthenticationResult( @@ -167,16 +168,15 @@ private PlannerResult planWithDruidConvention( ); } else { authResult = AuthorizationUtils.authorizeAllResourceActions( - datasourceNames, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, authenticationResult, + Iterables.transform(datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR), authorizerMapper ); plannerContext.setAuthenticationResult(authenticationResult); } if (!authResult.isAllowed()) { - throw new SecurityException(authResult.toString()); + throw new ForbiddenException(authResult.toString()); } if (explain != null) { @@ -246,16 +246,14 @@ public void visit(RelNode node, int ordinal, RelNode parent) ); return AuthorizationUtils.authorizeAllResourceActions( req, - datasourceNames, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, + Iterables.transform(datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR), authorizerMapper ); } else { plannerContext.setAuthenticationResult(authenticationResult); return AuthorizationUtils.authorizeAllResourceActions( - datasourceNames, - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR, authenticationResult, + Iterables.transform(datasourceNames, AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR), authorizerMapper ); } @@ -294,7 +292,7 @@ private PlannerResult planWithBindableConvention( Access accessResult = authorizeBindableRel(bindableRel, plannerContext, request, authenticationResult); if (!accessResult.isAllowed()) { - throw new SecurityException(accessResult.toString()); + throw new ForbiddenException(accessResult.toString()); } if (explain != null) { diff --git a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java index 4afdb419879e..e2e2a8595baa 100644 --- a/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java +++ b/sql/src/main/java/io/druid/sql/calcite/rel/DruidQueryBuilder.java @@ -638,7 +638,7 @@ public ScanQuery toScanQuery( filtration.getDimFilter(), Ordering.natural().sortedCopy(ImmutableSet.copyOf(getRowOrder())), false, - plannerContext.getQueryContext() + ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) ); } diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java index d0877cbbf93d..75e75f335199 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -42,13 +42,8 @@ import io.druid.math.expr.ExprMacroTable; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -84,7 +79,6 @@ import java.sql.Timestamp; import java.sql.Types; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -135,10 +129,6 @@ public void setUp() throws Exception final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); - final Map testAuthorizerMap = new HashMap<>(); - testAuthorizerMap.put("allowAll", new AllowAllAuthorizer()); - final Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), @@ -163,8 +153,8 @@ public void configure(Binder binder) macroTable, plannerConfig, new AuthConfig(), - new AuthenticatorMapper(defaultMap, "allowAll"), - new AuthorizerMapper(testAuthorizerMap), + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ), AVATICA_CONFIG, @@ -598,10 +588,6 @@ public int getMaxRowsPerFrame() final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); - final Map testAuthorizerMap = new HashMap<>(); - testAuthorizerMap.put("allowAll", new AllowAllAuthorizer()); - final Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); final List frames = new ArrayList<>(); DruidMeta smallFrameDruidMeta = new DruidMeta( new PlannerFactory( @@ -611,8 +597,8 @@ public int getMaxRowsPerFrame() macroTable, plannerConfig, new AuthConfig(), - new AuthenticatorMapper(defaultMap, "allowAll"), - new AuthorizerMapper(testAuthorizerMap), + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ), smallFrameConfig, diff --git a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java index d6c6d9cb07d3..7cd37b3ade77 100644 --- a/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/io/druid/sql/avatica/DruidStatementTest.java @@ -21,16 +21,12 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; + import io.druid.java.util.common.DateTimes; import io.druid.math.expr.ExprMacroTable; import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -49,7 +45,6 @@ import org.junit.rules.TemporaryFolder; import java.util.List; -import java.util.Map; public class DruidStatementTest { @@ -74,8 +69,6 @@ public void setUp() throws Exception ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); - final Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); plannerFactory = new PlannerFactory( druidSchema, CalciteTests.createMockQueryLifecycleFactory(walker), @@ -83,14 +76,8 @@ public void setUp() throws Exception macroTable, plannerConfig, new AuthConfig(), - new AuthenticatorMapper(defaultMap, "allowAll"), - new AuthorizerMapper(null) { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }, + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ); } diff --git a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java index f3a3d799a61d..9839fb93e80f 100644 --- a/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/CalciteQueryTest.java @@ -87,13 +87,8 @@ import io.druid.segment.column.Column; import io.druid.segment.column.ValueType; import io.druid.segment.virtual.ExpressionVirtualColumn; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.filtration.Filtration; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -455,7 +450,7 @@ public void testExplainSelectStar() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" + "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); @@ -710,8 +705,8 @@ public void testExplainSelfJoinWithFallback() throws Exception new Object[]{ "BindableProject(dim1=[$9], dim10=[$2], dim2=[$3])\n" + " BindableJoin(condition=[=($9, $3)], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807},\"descending\":false}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"\",\"extractionFn\":null}},\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"},\"descending\":false}])\n" } ) ); @@ -5520,8 +5515,6 @@ private List getResults( final DruidSchema druidSchema = CalciteTests.createMockSchema(walker, plannerConfig, viewManager); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); - final Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); final PlannerFactory plannerFactory = new PlannerFactory( druidSchema, @@ -5530,14 +5523,8 @@ private List getResults( macroTable, plannerConfig, new AuthConfig(), - new AuthenticatorMapper(defaultMap, "allowAll"), - new AuthorizerMapper(null) { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }, + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ); diff --git a/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java index ff73201d2770..e27b9eee219f 100644 --- a/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/expression/ExpressionsTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.PeriodGranularity; @@ -30,9 +29,7 @@ import io.druid.query.extraction.RegexDimExtractionFn; import io.druid.query.extraction.TimeFormatExtractionFn; import io.druid.segment.column.ValueType; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.planner.PlannerContext; @@ -63,16 +60,11 @@ public class ExpressionsTest { private static final DateTimeZone LOS_ANGELES = DateTimeZone.forID("America/Los_Angeles"); - private static final Map defaultMap = Maps.newHashMap(); - { - defaultMap.put("allowAll", new AllowAllAuthenticator()); - } - private final PlannerContext plannerContext = PlannerContext.create( CalciteTests.createOperatorTable(), CalciteTests.createExprMacroTable(), new PlannerConfig(), - new AuthorizerMapper(null), + AuthTestUtils.TEST_AUTHORIZER_MAPPER, ImmutableMap.of() ); private final RowSignature rowSignature = RowSignature diff --git a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java index 9d5047d001ad..772a3adffe20 100644 --- a/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/http/SqlResourceTest.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; @@ -31,12 +30,8 @@ import io.druid.query.QueryInterruptedException; import io.druid.query.ResourceLimitExceededException; import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.DruidOperatorTable; import io.druid.sql.calcite.planner.PlannerConfig; @@ -91,6 +86,9 @@ public void setUp() throws Exception final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); req = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) .andReturn(AllowAllAuthenticator.ALLOW_ALL_RESULT) .anyTimes(); @@ -101,19 +99,6 @@ public void setUp() throws Exception .anyTimes(); EasyMock.replay(req); - - AuthorizerMapper authorizerMapper = new AuthorizerMapper(null) { - @Override - public Authorizer getAuthorizer(String name) - { - return new AllowAllAuthorizer(); - } - }; - - Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); - AuthenticatorMapper authenticatorMapper = new AuthenticatorMapper(defaultMap, "allowAll"); - resource = new SqlResource( JSON_MAPPER, new PlannerFactory( @@ -123,8 +108,8 @@ public Authorizer getAuthorizer(String name) macroTable, plannerConfig, new AuthConfig(), - authenticatorMapper, - authorizerMapper, + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER, + AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper() ) ); diff --git a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java index b03b742be75d..695646b1e3a3 100644 --- a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Maps; import io.druid.data.input.InputRow; import io.druid.java.util.common.Intervals; import io.druid.query.aggregation.CountAggregatorFactory; @@ -33,9 +32,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndexSchema; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; import io.druid.sql.calcite.table.DruidTable; @@ -143,15 +140,13 @@ public void setUp() throws Exception index2 ); - Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); schema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker), new TestServerInventoryView(walker.getSegments()), PLANNER_CONFIG_DEFAULT, new NoopViewManager(), - new AuthenticatorMapper(defaultMap, "allowAll") + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER ); schema.start(); diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index c64264d2b0c3..e9d12f65ccad 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -25,7 +25,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; @@ -93,13 +92,8 @@ import io.druid.server.QueryLifecycleFactory; import io.druid.server.initialization.ServerConfig; import io.druid.server.log.NoopRequestLogger; -import io.druid.server.security.AllowAllAuthenticator; -import io.druid.server.security.AllowAllAuthorizer; import io.druid.server.security.AuthConfig; -import io.druid.server.security.Authenticator; -import io.druid.server.security.AuthenticatorMapper; -import io.druid.server.security.Authorizer; -import io.druid.server.security.AuthorizerMapper; +import io.druid.server.security.AuthTestUtils; import io.druid.sql.calcite.aggregation.SqlAggregator; import io.druid.sql.calcite.expression.SqlOperatorConversion; import io.druid.sql.calcite.planner.DruidOperatorTable; @@ -116,7 +110,6 @@ import java.io.File; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -321,8 +314,6 @@ public static QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate() public static QueryLifecycleFactory createMockQueryLifecycleFactory(final QuerySegmentWalker walker) { - Map testAuthorizerMap = new HashMap<>(); - testAuthorizerMap.put("allowAll", new AllowAllAuthorizer()); return new QueryLifecycleFactory( new QueryToolChestWarehouse() { @@ -338,7 +329,7 @@ public > QueryToolChest getToolChest new NoopRequestLogger(), new ServerConfig(), new AuthConfig(), - new AuthorizerMapper(testAuthorizerMap) + AuthTestUtils.TEST_AUTHORIZER_MAPPER ); } @@ -427,15 +418,12 @@ public static DruidSchema createMockSchema( final ViewManager viewManager ) { - Map defaultMap = Maps.newHashMap(); - defaultMap.put("allowAll", new AllowAllAuthenticator()); - final DruidSchema schema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker), new TestServerInventoryView(walker.getSegments()), plannerConfig, viewManager, - new AuthenticatorMapper(defaultMap, "allowAll") + AuthTestUtils.TEST_AUTHENTICATOR_MAPPER ); schema.start(); From baae14c9e5749a44522088f88e5c69359207b3b0 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 15 Sep 2017 03:11:19 -0700 Subject: [PATCH 30/34] Fix test --- .../security/PreResponseAuthorizationCheckFilterTest.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index d71bd236f130..a67a5c922d1e 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -20,6 +20,8 @@ package io.druid.server.http.security; import com.google.common.collect.Lists; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.ISE; import io.druid.server.security.AllowAllAuthenticator; @@ -96,6 +98,8 @@ public void testAuthenticationFailedRequest() throws Exception @Test public void testMissingAuthorizationCheck() throws Exception { + EmittingLogger.registerEmitter(EasyMock.createNiceMock(ServiceEmitter.class)); + expectedException.expect(ISE.class); expectedException.expectMessage("Request did not have an authorization check performed: uri"); @@ -110,6 +114,7 @@ public void testMissingAuthorizationCheck() throws Exception EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).once(); EasyMock.expect(resp.getStatus()).andReturn(200).once(); EasyMock.expect(req.getRequestURI()).andReturn("uri").once(); + EasyMock.expect(resp.isCommitted()).andReturn(true).once(); resp.setStatus(403); EasyMock.expectLastCall().once(); resp.setContentType("application/json"); From 99a4972ecbeae3d30d9190ed7170242237e855f2 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 15 Sep 2017 10:29:45 -0700 Subject: [PATCH 31/34] Fix IT --- .../src/main/java/io/druid/server/http/DatasourcesResource.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/io/druid/server/http/DatasourcesResource.java b/server/src/main/java/io/druid/server/http/DatasourcesResource.java index 93d9393a8008..e473ca2b0082 100644 --- a/server/src/main/java/io/druid/server/http/DatasourcesResource.java +++ b/server/src/main/java/io/druid/server/http/DatasourcesResource.java @@ -249,6 +249,7 @@ public Response deleteDataSource( @DELETE @Path("/{dataSourceName}/intervals/{interval}") + @ResourceFilters(DatasourceResourceFilter.class) @Produces(MediaType.APPLICATION_JSON) public Response deleteDataSourceSpecificInterval( @PathParam("dataSourceName") final String dataSourceName, From e833bef59a68694d8d32d3d32474393440bb43f2 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 15 Sep 2017 13:20:07 -0700 Subject: [PATCH 32/34] More PR comments --- .../DruidKerberosAuthenticationHandler.java | 4 +- .../java/io/druid/server/QueryLifecycle.java | 18 ++--- .../java/io/druid/server/QueryResource.java | 3 +- .../server/security/AuthenticationUtils.java | 4 +- .../AuthenticationWrappingFilter.java | 69 +++++++++++++++++++ .../server/security/ForbiddenException.java | 2 +- .../PreResponseAuthorizationCheckFilter.java | 59 +++++++++------- ...eResponseAuthorizationCheckFilterTest.java | 42 +++++------ 8 files changed, 136 insertions(+), 65 deletions(-) create mode 100644 server/src/main/java/io/druid/server/security/AuthenticationWrappingFilter.java diff --git a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java index 2a564104ef6e..50fe3b0f3a52 100644 --- a/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java +++ b/extensions-core/druid-kerberos/src/main/java/io/druid/security/kerberos/DruidKerberosAuthenticationHandler.java @@ -65,7 +65,7 @@ public void destroy() loginContext.logout(); } catch (LoginException ex) { - log.warn(ex.getMessage(), ex); + log.warn(ex, ex.getMessage()); } } loginContexts.clear(); @@ -117,7 +117,7 @@ public void init(Properties config) throws ServletException loginContext.login(); } catch (LoginException le) { - log.warn("Failed to login as [{}]", spnegoPrincipal, le); + log.warn(le, "Failed to login as [{}]", spnegoPrincipal); throw new AuthenticationException(le); } loginContexts.add(loginContext); diff --git a/server/src/main/java/io/druid/server/QueryLifecycle.java b/server/src/main/java/io/druid/server/QueryLifecycle.java index a892c06421c2..8785397bb3c8 100644 --- a/server/src/main/java/io/druid/server/QueryLifecycle.java +++ b/server/src/main/java/io/druid/server/QueryLifecycle.java @@ -111,20 +111,16 @@ public QueryLifecycle( * is unauthorized, an IllegalStateException will be thrown. Logs and metrics are emitted when the Sequence is * either fully iterated or throws an exception. * - * @param query the query - * @param user authentication token from the request - * @param namespace authentication namespace of the request - * @param remoteAddress remote address, for logging; or null if unknown - * @param needsAuth if false, skip the authorization check. This is useful when the authorization check has - * already been performed (e.g. in SQL handling, where authorization takes place in the - * planning step) + * @param query the query + * @param authenticationResult authentication result indicating identity of the requester + * @param remoteAddress remote address, for logging; or null if unknown * * @return results */ @SuppressWarnings("unchecked") public Sequence runSimple( final Query query, - @Nullable final AuthenticationResult authenticationResult, + final AuthenticationResult authenticationResult, @Nullable final String remoteAddress ) { @@ -186,15 +182,13 @@ public void initialize(final Query baseQuery) /** * Authorize the query. Will return an Access object denoting whether the query is authorized or not. * - * @param token authentication token from the request - * @param namespace namespace of the authentication token - * @param authenticationResult authentication result of the request + * @param authenticationResult authentication result indicating the identity of the requester * * @return authorization result * * */ public Access authorize( - @Nullable final AuthenticationResult authenticationResult + final AuthenticationResult authenticationResult ) { transition(State.INITIALIZED, State.AUTHORIZING); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 2cda222e4655..bdf458dcb65d 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -275,8 +275,7 @@ public void write(OutputStream outputStream) throws IOException, WebApplicationE catch (ForbiddenException e) { // don't do anything for an authorization failure, ForbiddenExceptionMapper will catch this later and // send an error response if this is thrown. - Throwables.propagate(e); - return context.gotError(e); + throw e; } catch (Exception e) { failedQueryCount.incrementAndGet(); diff --git a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java index 62697885ef0c..cabaa828274c 100644 --- a/server/src/main/java/io/druid/server/security/AuthenticationUtils.java +++ b/server/src/main/java/io/druid/server/security/AuthenticationUtils.java @@ -33,7 +33,9 @@ public static void addAuthenticationFilterChain( ) { for (Authenticator authenticator : authenticators) { - FilterHolder holder = new FilterHolder(authenticator.getFilter()); + FilterHolder holder = new FilterHolder( + new AuthenticationWrappingFilter(authenticator.getFilter()) + ); if (authenticator.getInitParameters() != null) { holder.setInitParameters(authenticator.getInitParameters()); } diff --git a/server/src/main/java/io/druid/server/security/AuthenticationWrappingFilter.java b/server/src/main/java/io/druid/server/security/AuthenticationWrappingFilter.java new file mode 100644 index 000000000000..c5ce96374350 --- /dev/null +++ b/server/src/main/java/io/druid/server/security/AuthenticationWrappingFilter.java @@ -0,0 +1,69 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.security; + +import javax.servlet.Filter; +import javax.servlet.FilterChain; +import javax.servlet.FilterConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import java.io.IOException; + +/** + * Used to wrap Filters created by Authenticators, this wrapper filter skips itself if a request already + * has an authentication check (so that Authenticator implementations don't have to perform this check themselves) + */ +public class AuthenticationWrappingFilter implements Filter +{ + private final Filter delegate; + + public AuthenticationWrappingFilter( + final Filter delegate + ) + { + this.delegate = delegate; + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException + { + delegate.init(filterConfig); + } + + @Override + public void doFilter( + ServletRequest request, ServletResponse response, FilterChain chain + ) throws IOException, ServletException + { + // If there's already an auth result, then we have authenticated already, skip this. + if (request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) != null) { + chain.doFilter(request, response); + } else { + delegate.doFilter(request, response, chain); + } + } + + @Override + public void destroy() + { + delegate.destroy(); + } +} diff --git a/server/src/main/java/io/druid/server/security/ForbiddenException.java b/server/src/main/java/io/druid/server/security/ForbiddenException.java index a317a8b94447..fc7324cb7013 100644 --- a/server/src/main/java/io/druid/server/security/ForbiddenException.java +++ b/server/src/main/java/io/druid/server/security/ForbiddenException.java @@ -23,7 +23,7 @@ * Throw this when a request is unauthorized and we want to send a 403 response back, Jersey exception mapper will * take care of sending the response. */ -public class ForbiddenException extends SecurityException +public class ForbiddenException extends RuntimeException { public ForbiddenException() { diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index 77b48a116d6b..d63b6fc0c9a2 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -76,30 +76,8 @@ public void doFilter( { final HttpServletResponse response = (HttpServletResponse) servletResponse; - // Since this is the last filter in the chain, some previous authentication filter - // should have placed an authentication result in the request. - // If not, send an authentication challenge. if (servletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) == null) { - Set supportedAuthSchemes = Sets.newHashSet(); - for (Authenticator authenticator : authenticators) { - String challengeHeader = authenticator.getAuthChallengeHeader(); - if (challengeHeader != null) { - supportedAuthSchemes.add(challengeHeader); - } - } - for (String authScheme : supportedAuthSchemes) { - response.addHeader("WWW-Authenticate", authScheme); - } - QueryInterruptedException unauthorizedError = new QueryInterruptedException( - QueryInterruptedException.UNAUTHORIZED, - null, - null, - DruidNode.getDefaultHost() - ); - unauthorizedError.setStackTrace(new StackTraceElement[0]); - OutputStream out = servletResponse.getOutputStream(); - sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); - out.close(); + handleUnauthenticatedRequest((HttpServletRequest) servletRequest, response); return; } @@ -132,12 +110,41 @@ public void destroy() } - private void handleAuthorizationCheckError(String errorMsg, ServletResponse servletResponse) + private void handleUnauthenticatedRequest( + final HttpServletRequest request, + final HttpServletResponse response + ) throws IOException { - log.error(errorMsg); + // Since this is the last filter in the chain, some previous authentication filter + // should have placed an authentication result in the request. + // If not, send an authentication challenge. + Set supportedAuthSchemes = Sets.newHashSet(); + for (Authenticator authenticator : authenticators) { + String challengeHeader = authenticator.getAuthChallengeHeader(); + if (challengeHeader != null) { + supportedAuthSchemes.add(challengeHeader); + } + } + for (String authScheme : supportedAuthSchemes) { + response.addHeader("WWW-Authenticate", authScheme); + } + QueryInterruptedException unauthorizedError = new QueryInterruptedException( + QueryInterruptedException.UNAUTHORIZED, + null, + null, + DruidNode.getDefaultHost() + ); + unauthorizedError.setStackTrace(new StackTraceElement[0]); + OutputStream out = response.getOutputStream(); + sendJsonError(response, Response.SC_UNAUTHORIZED, jsonMapper.writeValueAsString(unauthorizedError), out); + out.close(); + return; + } + private void handleAuthorizationCheckError(String errorMsg, ServletResponse servletResponse) + { // Send out an alert so there's a centralized collection point for seeing errors of this nature - log.makeAlert(errorMsg); + log.makeAlert(errorMsg).emit(); if (servletResponse.isCommitted()) { throw new ISE(errorMsg); diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index a67a5c922d1e..e90912c2dadf 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -72,27 +72,27 @@ public void testValidRequest() throws Exception @Test public void testAuthenticationFailedRequest() throws Exception { - HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); - HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); - FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); - ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); - - EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); - EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).once(); - resp.setStatus(401); - EasyMock.expectLastCall().once(); - resp.setContentType("application/json"); - EasyMock.expectLastCall().once(); - resp.setCharacterEncoding("UTF-8"); - EasyMock.expectLastCall().once(); - EasyMock.replay(req, resp, filterChain, outputStream); - - PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( - authenticators, - new DefaultObjectMapper() - ); - filter.doFilter(req, resp, filterChain); - EasyMock.verify(req, resp, filterChain, outputStream); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + HttpServletResponse resp = EasyMock.createStrictMock(HttpServletResponse.class); + FilterChain filterChain = EasyMock.createNiceMock(FilterChain.class); + ServletOutputStream outputStream = EasyMock.createNiceMock(ServletOutputStream.class); + + EasyMock.expect(resp.getOutputStream()).andReturn(outputStream).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn(null).once(); + resp.setStatus(401); + EasyMock.expectLastCall().once(); + resp.setContentType("application/json"); + EasyMock.expectLastCall().once(); + resp.setCharacterEncoding("UTF-8"); + EasyMock.expectLastCall().once(); + EasyMock.replay(req, resp, filterChain, outputStream); + + PreResponseAuthorizationCheckFilter filter = new PreResponseAuthorizationCheckFilter( + authenticators, + new DefaultObjectMapper() + ); + filter.doFilter(req, resp, filterChain); + EasyMock.verify(req, resp, filterChain, outputStream); } @Test From 0f44de3040acc9ac04898990ab6afcda430dfac4 Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 15 Sep 2017 16:02:53 -0700 Subject: [PATCH 33/34] PR comments --- docs/content/configuration/auth.md | 4 ++- .../PreResponseAuthorizationCheckFilter.java | 36 +++++++++++-------- ...eResponseAuthorizationCheckFilterTest.java | 3 +- 3 files changed, 26 insertions(+), 17 deletions(-) diff --git a/docs/content/configuration/auth.md b/docs/content/configuration/auth.md index 29d32e502e55..d498d217f4fe 100644 --- a/docs/content/configuration/auth.md +++ b/docs/content/configuration/auth.md @@ -23,7 +23,9 @@ For example, the following authentication chain definition enables the Kerberos druid.auth.authenticationChain=["kerberos", "basic"] ``` -A request will pass through all Authenticators in the chain, unless one of the Authenticators sends an HTTP error response. If no Authenticator in the chain successfully authenticated a request, an HTTP error response will be sent. +A request will pass through all Authenticators in the chain, until one of the Authenticators successfully authenticates the request or sends an HTTP error response. Authenticators later in the chain will be skipped after the first successful authentication or if the request is terminated with an error response. + +If no Authenticator in the chain successfully authenticated a request or sent an HTTP error response, an HTTP error response will be sent at the end of the chain. Druid includes a built-in Authenticator, used for the default unsecured configuration. diff --git a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java index d63b6fc0c9a2..6743af04c204 100644 --- a/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java +++ b/server/src/main/java/io/druid/server/security/PreResponseAuthorizationCheckFilter.java @@ -23,7 +23,6 @@ import com.google.common.collect.Sets; import com.metamx.emitter.EmittingLogger; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.StringUtils; import io.druid.query.QueryInterruptedException; import io.druid.server.DruidNode; import org.eclipse.jetty.server.Response; @@ -75,9 +74,10 @@ public void doFilter( ) throws IOException, ServletException { final HttpServletResponse response = (HttpServletResponse) servletResponse; + final HttpServletRequest request = (HttpServletRequest) servletRequest; if (servletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT) == null) { - handleUnauthenticatedRequest((HttpServletRequest) servletRequest, response); + handleUnauthenticatedRequest(response); return; } @@ -85,22 +85,22 @@ public void doFilter( Boolean authInfoChecked = (Boolean) servletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED); if (authInfoChecked == null && !errorOverridesMissingAuth(response.getStatus())) { - String errorMsg = StringUtils.format( - "Request did not have an authorization check performed: %s", - ((HttpServletRequest) servletRequest).getRequestURI() - ); // Note: rather than throwing an exception here, it would be nice to blank out the original response // since the request didn't have any authorization checks performed. However, this breaks proxying // (e.g. OverlordServletProxy), so this is not implemented for now. - handleAuthorizationCheckError(errorMsg, servletResponse); + handleAuthorizationCheckError( + "Request did not have an authorization check performed.", + request, + response + ); } if (authInfoChecked != null && !authInfoChecked && response.getStatus() != Response.SC_FORBIDDEN) { - String errorMsg = StringUtils.format( - "Request's authorization check failed but status code was not 403: %s", - ((HttpServletRequest) servletRequest).getRequestURI() + handleAuthorizationCheckError( + "Request's authorization check failed but status code was not 403.", + request, + response ); - handleAuthorizationCheckError(errorMsg, servletResponse); } } @@ -111,7 +111,6 @@ public void destroy() } private void handleUnauthenticatedRequest( - final HttpServletRequest request, final HttpServletResponse response ) throws IOException { @@ -141,16 +140,23 @@ private void handleUnauthenticatedRequest( return; } - private void handleAuthorizationCheckError(String errorMsg, ServletResponse servletResponse) + private void handleAuthorizationCheckError( + String errorMsg, + HttpServletRequest servletRequest, + HttpServletResponse servletResponse + ) { // Send out an alert so there's a centralized collection point for seeing errors of this nature - log.makeAlert(errorMsg).emit(); + log.makeAlert(errorMsg) + .addData("uri", servletRequest.getRequestURI()) + .addData("method", servletRequest.getMethod()) + .emit(); if (servletResponse.isCommitted()) { throw new ISE(errorMsg); } else { try { - ((HttpServletResponse) servletResponse).sendError(Response.SC_FORBIDDEN); + servletResponse.sendError(Response.SC_FORBIDDEN); } catch (Exception e) { throw new RuntimeException(e); diff --git a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java index e90912c2dadf..e9cf0a23fa4b 100644 --- a/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java +++ b/server/src/test/java/io/druid/server/http/security/PreResponseAuthorizationCheckFilterTest.java @@ -101,7 +101,7 @@ public void testMissingAuthorizationCheck() throws Exception EmittingLogger.registerEmitter(EasyMock.createNiceMock(ServiceEmitter.class)); expectedException.expect(ISE.class); - expectedException.expectMessage("Request did not have an authorization check performed: uri"); + expectedException.expectMessage("Request did not have an authorization check performed."); AuthenticationResult authenticationResult = new AuthenticationResult("so-very-valid", "so-very-valid"); @@ -114,6 +114,7 @@ public void testMissingAuthorizationCheck() throws Exception EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).once(); EasyMock.expect(resp.getStatus()).andReturn(200).once(); EasyMock.expect(req.getRequestURI()).andReturn("uri").once(); + EasyMock.expect(req.getMethod()).andReturn("GET").once(); EasyMock.expect(resp.isCommitted()).andReturn(true).once(); resp.setStatus(403); EasyMock.expectLastCall().once(); From 9724acc8dd0eec2defddab18b28072af77a31eeb Mon Sep 17 00:00:00 2001 From: jon-wei Date: Fri, 15 Sep 2017 20:02:02 -0700 Subject: [PATCH 34/34] SSL fix --- .../src/main/java/io/druid/https/SSLContextModule.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextModule.java b/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextModule.java index 9de19dbbd3e2..b2de27468d7d 100644 --- a/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextModule.java +++ b/extensions-core/simple-client-sslcontext/src/main/java/io/druid/https/SSLContextModule.java @@ -24,6 +24,8 @@ import com.google.inject.Binder; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Client; +import io.druid.guice.annotations.EscalatedClient; +import io.druid.guice.annotations.EscalatedGlobal; import io.druid.guice.annotations.Global; import io.druid.initialization.DruidModule; import io.druid.server.router.Router; @@ -48,5 +50,7 @@ public void configure(Binder binder) binder.bind(SSLContext.class).annotatedWith(Global.class).toProvider(SSLContextProvider.class); binder.bind(SSLContext.class).annotatedWith(Client.class).toProvider(SSLContextProvider.class); binder.bind(SSLContext.class).annotatedWith(Router.class).toProvider(SSLContextProvider.class); + binder.bind(SSLContext.class).annotatedWith(EscalatedGlobal.class).toProvider(SSLContextProvider.class); + binder.bind(SSLContext.class).annotatedWith(EscalatedClient.class).toProvider(SSLContextProvider.class); } }