Skip to content

Commit

Permalink
Add support for pluggable Custom Presto Authenticators
Browse files Browse the repository at this point in the history
Co-authored-by: Namya Sehgal <sehgalnamya@gmail.com>
  • Loading branch information
imsayari404 authored and tdcmeehan committed Jan 10, 2025
1 parent 5d63246 commit 6b2683c
Show file tree
Hide file tree
Showing 13 changed files with 301 additions and 2 deletions.
1 change: 1 addition & 0 deletions presto-docs/src/main/sphinx/develop.rst
Original file line number Diff line number Diff line change
Expand Up @@ -20,3 +20,4 @@ This guide is intended for Presto contributors and plugin developers.
develop/worker-protocol
develop/serialized-page
develop/presto-console
develop/presto-authenticator
52 changes: 52 additions & 0 deletions presto-docs/src/main/sphinx/develop/presto-authenticator.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
===========================
Custom Presto Authenticator
===========================

Presto supports authentication through a custom Presto authenticator
that validates the request and creates a principal.

Implementation
--------------

``PrestoAuthenticatorFactory`` creates a
``PrestoAuthenticator`` instance. It also defines the name of this
authenticator which is used by the administrator in a Presto configuration.

``PrestoAuthenticator`` contains a single method, ``createAuthenticatedPrincipal()``,
that validates the request and returns a ``Principal``, which is then
authorized by the :doc:`system-access-control`.

The implementation of ``PrestoAuthenticatorFactory`` must be wrapped
as a plugin and installed on the Presto cluster.

Configuration
-------------

After a plugin that implements ``PrestoAuthenticatorFactory`` has been
installed on the coordinator, it is configured using an
``etc/presto-authenticator.properties`` file. All of the
properties other than ``presto-authenticator.name`` are specific to the
``PrestoAuthenticatorFactory`` implementation.

The ``presto-authenticator.name`` property is used by Presto to find a
registered ``PrestoAuthenticatorFactory`` based on the name returned by
``PrestoAuthenticatorFactory.getName()``. The remaining properties are
passed as a map to ``PrestoAuthenticatorFactory.create()``.

Example configuration file:

.. code-block:: none
presto-authenticator.name=custom-authenticator
custom-property1=custom-value1
custom-property2=custom-value2
Additionally, the coordinator must be configured to use custom authentication
and have HTTPS enabled.

Add the property shown below to the coordinator's ``config.properties`` file:

.. code-block:: none
http-server.authentication.type=CUSTOM
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.security.AccessControlManager;
import com.facebook.presto.server.security.PasswordAuthenticatorManager;
import com.facebook.presto.server.security.PrestoAuthenticatorManager;
import com.facebook.presto.spi.CoordinatorPlugin;
import com.facebook.presto.spi.Plugin;
import com.facebook.presto.spi.analyzer.AnalyzerProvider;
Expand All @@ -40,6 +41,7 @@
import com.facebook.presto.spi.prerequisites.QueryPrerequisitesFactory;
import com.facebook.presto.spi.resourceGroups.ResourceGroupConfigurationManagerFactory;
import com.facebook.presto.spi.security.PasswordAuthenticatorFactory;
import com.facebook.presto.spi.security.PrestoAuthenticatorFactory;
import com.facebook.presto.spi.security.SystemAccessControlFactory;
import com.facebook.presto.spi.session.SessionPropertyConfigurationManagerFactory;
import com.facebook.presto.spi.session.WorkerSessionPropertyProviderFactory;
Expand Down Expand Up @@ -116,6 +118,7 @@ public class PluginManager
private final ResourceGroupManager<?> resourceGroupManager;
private final AccessControlManager accessControlManager;
private final PasswordAuthenticatorManager passwordAuthenticatorManager;
private final PrestoAuthenticatorManager prestoAuthenticatorManager;
private final EventListenerManager eventListenerManager;
private final BlockEncodingManager blockEncodingManager;
private final TempStorageManager tempStorageManager;
Expand Down Expand Up @@ -147,6 +150,7 @@ public PluginManager(
QueryPreparerProviderManager queryPreparerProviderManager,
AccessControlManager accessControlManager,
PasswordAuthenticatorManager passwordAuthenticatorManager,
PrestoAuthenticatorManager prestoAuthenticatorManager,
EventListenerManager eventListenerManager,
BlockEncodingManager blockEncodingManager,
TempStorageManager tempStorageManager,
Expand Down Expand Up @@ -176,6 +180,7 @@ public PluginManager(
this.resourceGroupManager = requireNonNull(resourceGroupManager, "resourceGroupManager is null");
this.accessControlManager = requireNonNull(accessControlManager, "accessControlManager is null");
this.passwordAuthenticatorManager = requireNonNull(passwordAuthenticatorManager, "passwordAuthenticatorManager is null");
this.prestoAuthenticatorManager = requireNonNull(prestoAuthenticatorManager, "prestoAuthenticatorManager is null");
this.eventListenerManager = requireNonNull(eventListenerManager, "eventListenerManager is null");
this.blockEncodingManager = requireNonNull(blockEncodingManager, "blockEncodingManager is null");
this.tempStorageManager = requireNonNull(tempStorageManager, "tempStorageManager is null");
Expand Down Expand Up @@ -305,6 +310,11 @@ public void installPlugin(Plugin plugin)
passwordAuthenticatorManager.addPasswordAuthenticatorFactory(authenticatorFactory);
}

for (PrestoAuthenticatorFactory authenticatorFactory : plugin.getPrestoAuthenticatorFactories()) {
log.info("Registering presto authenticator %s", authenticatorFactory.getName());
prestoAuthenticatorManager.addPrestoAuthenticatorFactory(authenticatorFactory);
}

for (EventListenerFactory eventListenerFactory : plugin.getEventListenerFactories()) {
log.info("Registering event listener %s", eventListenerFactory.getName());
eventListenerManager.addEventListenerFactory(eventListenerFactory);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import com.facebook.presto.security.AccessControlManager;
import com.facebook.presto.security.AccessControlModule;
import com.facebook.presto.server.security.PasswordAuthenticatorManager;
import com.facebook.presto.server.security.PrestoAuthenticatorManager;
import com.facebook.presto.server.security.ServerSecurityModule;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
import com.facebook.presto.sql.parser.SqlParserOptions;
Expand Down Expand Up @@ -175,6 +176,7 @@ public void run()
injector.getInstance(AccessControlManager.class).loadSystemAccessControl();
}
injector.getInstance(PasswordAuthenticatorManager.class).loadPasswordAuthenticator();
injector.getInstance(PrestoAuthenticatorManager.class).loadPrestoAuthenticator();
injector.getInstance(EventListenerManager.class).loadConfiguredEventListener();
injector.getInstance(TempStorageManager.class).loadTempStorages();
injector.getInstance(QueryPrerequisitesManager.class).loadQueryPrerequisites();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.server.security;

import com.facebook.airlift.http.server.AuthenticationException;
import com.facebook.airlift.http.server.Authenticator;
import com.facebook.presto.spi.security.AccessDeniedException;

import javax.inject.Inject;
import javax.servlet.http.HttpServletRequest;

import java.security.Principal;
import java.util.List;
import java.util.Map;

import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static java.util.Collections.list;
import static java.util.Objects.requireNonNull;

public class CustomPrestoAuthenticator
implements Authenticator
{
private PrestoAuthenticatorManager authenticatorManager;

@Inject
public CustomPrestoAuthenticator(PrestoAuthenticatorManager authenticatorManager)
{
this.authenticatorManager = requireNonNull(authenticatorManager, "authenticatorManager is null");
}

@Override
public Principal authenticate(HttpServletRequest request)
throws AuthenticationException
{
try {
// Extracting headers into a Map
Map<String, List<String>> headers = getHeadersMap(request);

// Passing the header map to the authenticator (instead of HttpServletRequest)
return authenticatorManager.getAuthenticator().createAuthenticatedPrincipal(headers);
}
catch (AccessDeniedException e) {
throw new AuthenticationException(e.getMessage());
}
}

// Utility method to extract headers from HttpServletRequest
private Map<String, List<String>> getHeadersMap(HttpServletRequest request)
{
return list(request.getHeaderNames())
.stream()
.collect(toImmutableMap(
headerName -> headerName,
headerName -> list(request.getHeaders(headerName))));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.server.security;

import com.facebook.airlift.log.Logger;
import com.facebook.presto.spi.security.PrestoAuthenticator;
import com.facebook.presto.spi.security.PrestoAuthenticatorFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;

import javax.inject.Inject;

import java.io.File;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicReference;

import static com.facebook.presto.util.PropertiesUtil.loadProperties;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Strings.isNullOrEmpty;
import static java.util.Objects.requireNonNull;

public class PrestoAuthenticatorManager
{
private static final Logger log = Logger.get(PrestoAuthenticatorManager.class);

private static final File CONFIG_FILE = new File("etc/presto-authenticator.properties");
private static final String NAME_PROPERTY = "presto-authenticator.name";

private final Map<String, PrestoAuthenticatorFactory> factories = new ConcurrentHashMap<>();
private final AtomicReference<PrestoAuthenticator> authenticator = new AtomicReference<>();
private final boolean customAuthenticatorRequested;

@Inject
public PrestoAuthenticatorManager(SecurityConfig securityConfig)
{
this.customAuthenticatorRequested = securityConfig.getAuthenticationTypes().contains(SecurityConfig.AuthenticationType.CUSTOM);
}

public void addPrestoAuthenticatorFactory(PrestoAuthenticatorFactory factory)
{
checkArgument(factories.putIfAbsent(factory.getName(), factory) == null,
"Presto authenticator '%s' is already registered", factory.getName());
}

@VisibleForTesting
public void loadAuthenticator(String authenticatorName)
{
PrestoAuthenticatorFactory factory = factories.get(authenticatorName);

PrestoAuthenticator authenticator = factory.create(ImmutableMap.of());
this.authenticator.set(requireNonNull(authenticator, "authenticator is null"));
}

public void loadPrestoAuthenticator()
throws Exception
{
if (!customAuthenticatorRequested) {
return;
}

File configFileLocation = CONFIG_FILE.getAbsoluteFile();
Map<String, String> properties = new HashMap<>(loadProperties(configFileLocation));

String name = properties.remove(NAME_PROPERTY);
checkArgument(!isNullOrEmpty(name),
"Presto authenticator configuration %s does not contain %s", configFileLocation, NAME_PROPERTY);

log.info("-- Loading Presto authenticator --");

PrestoAuthenticatorFactory factory = factories.get(name);
checkState(factory != null, "Presto authenticator %s is not registered", name);

PrestoAuthenticator authenticator = factory.create(ImmutableMap.copyOf(properties));
this.authenticator.set(requireNonNull(authenticator, "authenticator is null"));

log.info("-- Loaded Presto authenticator %s --", name);
}

public PrestoAuthenticator getAuthenticator()
{
checkState(authenticator.get() != null, "authenticator was not loaded");
return authenticator.get();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,8 @@ public enum AuthenticationType
CERTIFICATE,
KERBEROS,
PASSWORD,
JWT
JWT,
CUSTOM
}

@NotNull
Expand All @@ -56,7 +57,7 @@ public SecurityConfig setAuthenticationTypes(List<AuthenticationType> authentica
}

@Config("http-server.authentication.type")
@ConfigDescription("Authentication types (supported types: CERTIFICATE, KERBEROS, PASSWORD, JWT)")
@ConfigDescription("Authentication types (supported types: CERTIFICATE, KERBEROS, PASSWORD, JWT, CUSTOM)")
public SecurityConfig setAuthenticationTypes(String types)
{
if (types == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@

import static com.facebook.airlift.configuration.ConfigBinder.configBinder;
import static com.facebook.presto.server.security.SecurityConfig.AuthenticationType.CERTIFICATE;
import static com.facebook.presto.server.security.SecurityConfig.AuthenticationType.CUSTOM;
import static com.facebook.presto.server.security.SecurityConfig.AuthenticationType.JWT;
import static com.facebook.presto.server.security.SecurityConfig.AuthenticationType.KERBEROS;
import static com.facebook.presto.server.security.SecurityConfig.AuthenticationType.PASSWORD;
Expand All @@ -39,6 +40,7 @@ public class ServerSecurityModule
protected void setup(Binder binder)
{
binder.bind(PasswordAuthenticatorManager.class).in(Scopes.SINGLETON);
binder.bind(PrestoAuthenticatorManager.class).in(Scopes.SINGLETON);

List<AuthenticationType> authTypes = buildConfigObject(SecurityConfig.class).getAuthenticationTypes();
Multibinder<Authenticator> authBinder = newSetBinder(binder, Authenticator.class);
Expand All @@ -58,6 +60,9 @@ else if (authType == JWT) {
configBinder(binder).bindConfig(JsonWebTokenConfig.class);
authBinder.addBinding().to(JsonWebTokenAuthenticator.class).in(Scopes.SINGLETON);
}
else if (authType == CUSTOM) {
authBinder.addBinding().to(CustomPrestoAuthenticator.class).in(Scopes.SINGLETON);
}
else {
throw new AssertionError("Unhandled auth type: " + authType);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,8 @@
import com.facebook.presto.server.PluginManagerConfig;
import com.facebook.presto.server.SessionPropertyDefaults;
import com.facebook.presto.server.security.PasswordAuthenticatorManager;
import com.facebook.presto.server.security.PrestoAuthenticatorManager;
import com.facebook.presto.server.security.SecurityConfig;
import com.facebook.presto.spi.ConnectorId;
import com.facebook.presto.spi.PageIndexerFactory;
import com.facebook.presto.spi.PageSorter;
Expand Down Expand Up @@ -521,6 +523,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig,
new QueryPreparerProviderManager(queryPreparerProvider),
accessControl,
new PasswordAuthenticatorManager(),
new PrestoAuthenticatorManager(new SecurityConfig()),
new EventListenerManager(),
blockEncodingManager,
new TestingTempStorageManager(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.facebook.presto.server.PluginManager;
import com.facebook.presto.server.SessionPropertyDefaults;
import com.facebook.presto.server.security.PasswordAuthenticatorManager;
import com.facebook.presto.server.security.PrestoAuthenticatorManager;
import com.facebook.presto.spark.classloader_interface.PrestoSparkBootstrapTimer;
import com.facebook.presto.spark.classloader_interface.SparkProcessType;
import com.facebook.presto.spi.security.AccessControl;
Expand Down Expand Up @@ -182,6 +183,7 @@ public Injector create(PrestoSparkBootstrapTimer bootstrapTimer)
injector.getInstance(StaticCatalogStore.class).loadCatalogs(catalogProperties);
injector.getInstance(ResourceGroupManager.class).loadConfigurationManager();
injector.getInstance(PasswordAuthenticatorManager.class).loadPasswordAuthenticator();
injector.getInstance(PrestoAuthenticatorManager.class).loadPrestoAuthenticator();
eventListenerProperties.ifPresent(properties -> injector.getInstance(EventListenerManager.class).loadConfiguredEventListener(properties));
bootstrapTimer.endSharedModulesLoading();

Expand Down
6 changes: 6 additions & 0 deletions presto-spi/src/main/java/com/facebook/presto/spi/Plugin.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import com.facebook.presto.spi.prerequisites.QueryPrerequisitesFactory;
import com.facebook.presto.spi.resourceGroups.ResourceGroupConfigurationManagerFactory;
import com.facebook.presto.spi.security.PasswordAuthenticatorFactory;
import com.facebook.presto.spi.security.PrestoAuthenticatorFactory;
import com.facebook.presto.spi.security.SystemAccessControlFactory;
import com.facebook.presto.spi.session.SessionPropertyConfigurationManagerFactory;
import com.facebook.presto.spi.statistics.HistoryBasedPlanStatisticsProvider;
Expand Down Expand Up @@ -75,6 +76,11 @@ default Iterable<PasswordAuthenticatorFactory> getPasswordAuthenticatorFactories
return emptyList();
}

default Iterable<PrestoAuthenticatorFactory> getPrestoAuthenticatorFactories()
{
return emptyList();
}

default Iterable<EventListenerFactory> getEventListenerFactories()
{
return emptyList();
Expand Down
Loading

0 comments on commit 6b2683c

Please sign in to comment.