diff --git a/docs/src/main/sphinx/connector/delta-lake.rst b/docs/src/main/sphinx/connector/delta-lake.rst index 6226931431ee..cc17059d0413 100644 --- a/docs/src/main/sphinx/connector/delta-lake.rst +++ b/docs/src/main/sphinx/connector/delta-lake.rst @@ -237,6 +237,37 @@ configure processing of Parquet files. * - ``parquet_writer_page_size`` - The maximum page size created by the Parquet writer. +.. _delta-lake-authorization: + +Authorization checks +^^^^^^^^^^^^^^^^^^^^ + +You can enable authorization checks for the connector by setting +the ``delta.security`` property in the catalog properties file. This +property must be one of the following values: + +.. list-table:: Delta Lake security values + :widths: 30, 60 + :header-rows: 1 + + * - Property value + - Description + * - ``ALLOW_ALL`` (default value) + - No authorization checks are enforced. + * - ``SYSTEM`` + - The connector relies on system-level access control. + * - ``READ_ONLY`` + - Operations that read data or metadata, such as :doc:`/sql/select` are + permitted. No operations that write data or metadata, such as + :doc:`/sql/create-table`, :doc:`/sql/insert`, or :doc:`/sql/delete` are + allowed. + * - ``FILE`` + - Authorization checks are enforced using a catalog-level access control + configuration file whose path is specified in the ``security.config-file`` + catalog configuration property. See + :ref:`catalog-file-based-access-control` for information on the + authorization configuration file. + .. _delta-lake-type-mapping: Type mapping diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeAccessControlMetadataFactory.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeAccessControlMetadataFactory.java deleted file mode 100644 index 84fd80e519ca..000000000000 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeAccessControlMetadataFactory.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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 io.trino.plugin.deltalake; - -import io.trino.plugin.hive.metastore.HiveMetastore; -import io.trino.plugin.hive.security.AccessControlMetadata; - -public interface DeltaLakeAccessControlMetadataFactory -{ - DeltaLakeAccessControlMetadataFactory SYSTEM = metastore -> new AccessControlMetadata() { - @Override - public boolean isUsingSystemSecurity() - { - return true; - } - }; - - AccessControlMetadata create(HiveMetastore metastore); -} diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java index 6954fcbd1a32..04792d1dfc5a 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadata.java @@ -56,11 +56,9 @@ import io.trino.plugin.hive.TableAlreadyExistsException; import io.trino.plugin.hive.metastore.Column; import io.trino.plugin.hive.metastore.Database; -import io.trino.plugin.hive.metastore.HivePrincipal; import io.trino.plugin.hive.metastore.PrincipalPrivileges; import io.trino.plugin.hive.metastore.StorageFormat; import io.trino.plugin.hive.metastore.Table; -import io.trino.plugin.hive.security.AccessControlMetadata; import io.trino.spi.NodeManager; import io.trino.spi.TrinoException; import io.trino.spi.block.Block; @@ -98,9 +96,6 @@ import io.trino.spi.expression.Variable; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; -import io.trino.spi.security.GrantInfo; -import io.trino.spi.security.Privilege; -import io.trino.spi.security.RoleGrant; import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.statistics.ColumnStatisticMetadata; import io.trino.spi.statistics.ColumnStatisticType; @@ -273,7 +268,6 @@ public class DeltaLakeMetadata private final TrinoFileSystemFactory fileSystemFactory; private final HdfsEnvironment hdfsEnvironment; private final TypeManager typeManager; - private final AccessControlMetadata accessControlMetadata; private final CheckpointWriterManager checkpointWriterManager; private final long defaultCheckpointInterval; private final boolean ignoreCheckpointWriteFailures; @@ -297,7 +291,6 @@ public DeltaLakeMetadata( TrinoFileSystemFactory fileSystemFactory, HdfsEnvironment hdfsEnvironment, TypeManager typeManager, - AccessControlMetadata accessControlMetadata, int domainCompactionThreshold, boolean unsafeWritesEnabled, JsonCodec dataFileInfoCodec, @@ -318,7 +311,6 @@ public DeltaLakeMetadata( this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); - this.accessControlMetadata = requireNonNull(accessControlMetadata, "accessControlMetadata is null"); this.domainCompactionThreshold = domainCompactionThreshold; this.unsafeWritesEnabled = unsafeWritesEnabled; this.dataFileInfoCodec = requireNonNull(dataFileInfoCodec, "dataFileInfoCodec is null"); @@ -2008,83 +2000,6 @@ public Map getSchemaProperties(ConnectorSession session, Catalog return db.map(DeltaLakeSchemaProperties::fromDatabase).orElseThrow(() -> new SchemaNotFoundException(schema)); } - @Override - public void createRole(ConnectorSession session, String role, Optional grantor) - { - accessControlMetadata.createRole(session, role, grantor.map(HivePrincipal::from)); - } - - @Override - public void dropRole(ConnectorSession session, String role) - { - accessControlMetadata.dropRole(session, role); - } - - @Override - public Set listRoles(ConnectorSession session) - { - return accessControlMetadata.listRoles(session); - } - - @Override - public Set listRoleGrants(ConnectorSession session, TrinoPrincipal principal) - { - return ImmutableSet.copyOf(accessControlMetadata.listRoleGrants(session, HivePrincipal.from(principal))); - } - - @Override - public void grantRoles(ConnectorSession session, Set roles, Set grantees, boolean withAdminOption, Optional grantor) - { - accessControlMetadata.grantRoles(session, roles, HivePrincipal.from(grantees), withAdminOption, grantor.map(HivePrincipal::from)); - } - - @Override - public void revokeRoles(ConnectorSession session, Set roles, Set grantees, boolean adminOptionFor, Optional grantor) - { - accessControlMetadata.revokeRoles(session, roles, HivePrincipal.from(grantees), adminOptionFor, grantor.map(HivePrincipal::from)); - } - - @Override - public Set listApplicableRoles(ConnectorSession session, TrinoPrincipal principal) - { - return accessControlMetadata.listApplicableRoles(session, HivePrincipal.from(principal)); - } - - @Override - public Set listEnabledRoles(ConnectorSession session) - { - return accessControlMetadata.listEnabledRoles(session); - } - - @Override - public void grantTablePrivileges(ConnectorSession session, SchemaTableName schemaTableName, Set privileges, TrinoPrincipal grantee, boolean grantOption) - { - accessControlMetadata.grantTablePrivileges(session, schemaTableName, privileges, HivePrincipal.from(grantee), grantOption); - } - - @Override - public void revokeTablePrivileges(ConnectorSession session, SchemaTableName schemaTableName, Set privileges, TrinoPrincipal grantee, boolean grantOption) - { - accessControlMetadata.revokeTablePrivileges(session, schemaTableName, privileges, HivePrincipal.from(grantee), grantOption); - } - - @Override - public List listTablePrivileges(ConnectorSession session, SchemaTablePrefix schemaTablePrefix) - { - return accessControlMetadata.listTablePrivileges(session, listTables(session, schemaTablePrefix)); - } - - private List listTables(ConnectorSession session, SchemaTablePrefix prefix) - { - if (prefix.getTable().isEmpty()) { - return listTables(session, prefix.getSchema()); - } - SchemaTableName tableName = prefix.toSchemaTableName(); - return metastore.getTable(tableName.getSchemaName(), tableName.getTableName()) - .map(table -> ImmutableList.of(tableName)) - .orElse(ImmutableList.of()); - } - private void setRollback(Runnable action) { checkState(rollbackAction.compareAndSet(null, action), "rollback action is already set"); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java index e8fd6841d953..4c0a999e1965 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMetadataFactory.java @@ -42,7 +42,6 @@ public class DeltaLakeMetadataFactory private final HdfsEnvironment hdfsEnvironment; private final TransactionLogAccess transactionLogAccess; private final TypeManager typeManager; - private final DeltaLakeAccessControlMetadataFactory accessControlMetadataFactory; private final JsonCodec dataFileInfoCodec; private final JsonCodec updateResultJsonCodec; private final JsonCodec mergeResultJsonCodec; @@ -68,7 +67,6 @@ public DeltaLakeMetadataFactory( HdfsEnvironment hdfsEnvironment, TransactionLogAccess transactionLogAccess, TypeManager typeManager, - DeltaLakeAccessControlMetadataFactory accessControlMetadataFactory, DeltaLakeConfig deltaLakeConfig, JsonCodec dataFileInfoCodec, JsonCodec updateResultJsonCodec, @@ -85,7 +83,6 @@ public DeltaLakeMetadataFactory( this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); this.transactionLogAccess = requireNonNull(transactionLogAccess, "transactionLogAccess is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); - this.accessControlMetadataFactory = requireNonNull(accessControlMetadataFactory, "accessControlMetadataFactory is null"); this.dataFileInfoCodec = requireNonNull(dataFileInfoCodec, "dataFileInfoCodec is null"); this.updateResultJsonCodec = requireNonNull(updateResultJsonCodec, "updateResultJsonCodec is null"); this.mergeResultJsonCodec = requireNonNull(mergeResultJsonCodec, "mergeResultJsonCodec is null"); @@ -123,7 +120,6 @@ public DeltaLakeMetadata create(ConnectorIdentity identity) fileSystemFactory, hdfsEnvironment, typeManager, - accessControlMetadataFactory.create(cachingHiveMetastore), domainCompactionThreshold, unsafeWritesEnabled, dataFileInfoCodec, diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeModule.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeModule.java index 67e1e0264720..946bd6707e1f 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeModule.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeModule.java @@ -96,8 +96,7 @@ public void setup(Binder binder) configBinder(binder).bindConfigDefaults(ParquetWriterConfig.class, config -> config.setParquetOptimizedWriterEnabled(true)); install(new ConnectorAccessControlModule()); - newOptionalBinder(binder, DeltaLakeAccessControlMetadataFactory.class) - .setDefault().toInstance(DeltaLakeAccessControlMetadataFactory.SYSTEM); + configBinder(binder).bindConfig(DeltaLakeSecurityConfig.class); Multibinder systemTableProviders = newSetBinder(binder, SystemTableProvider.class); systemTableProviders.addBinding().to(PropertiesSystemTableProvider.class).in(Scopes.SINGLETON); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSecurityConfig.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSecurityConfig.java new file mode 100644 index 000000000000..ef967b9c7344 --- /dev/null +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSecurityConfig.java @@ -0,0 +1,47 @@ +/* + * 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 io.trino.plugin.deltalake; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; + +import javax.validation.constraints.NotNull; + +import static io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity.ALLOW_ALL; + +public class DeltaLakeSecurityConfig +{ + public enum DeltaLakeSecurity + { + ALLOW_ALL, + READ_ONLY, + SYSTEM, + FILE, + } + + private DeltaLakeSecurity securitySystem = ALLOW_ALL; + + @NotNull + public DeltaLakeSecurity getSecuritySystem() + { + return securitySystem; + } + + @Config("delta.security") + @ConfigDescription("Authorization checks for Delta Lake connector") + public DeltaLakeSecurityConfig setSecuritySystem(DeltaLakeSecurity securitySystem) + { + this.securitySystem = securitySystem; + return this; + }} diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSecurityModule.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSecurityModule.java new file mode 100644 index 000000000000..226643973093 --- /dev/null +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSecurityModule.java @@ -0,0 +1,50 @@ +/* + * 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 io.trino.plugin.deltalake; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.trino.plugin.base.security.ConnectorAccessControlModule; +import io.trino.plugin.base.security.FileBasedAccessControlModule; +import io.trino.plugin.base.security.ReadOnlySecurityModule; +import io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity; +import io.trino.plugin.hive.security.AllowAllSecurityModule; + +import static io.airlift.configuration.ConditionalModule.conditionalModule; +import static io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity.ALLOW_ALL; +import static io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity.FILE; +import static io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity.READ_ONLY; + +public class DeltaLakeSecurityModule + extends AbstractConfigurationAwareModule +{ + @Override + protected void setup(Binder binder) + { + install(new ConnectorAccessControlModule()); + bindSecurityModule(ALLOW_ALL, new AllowAllSecurityModule()); + bindSecurityModule(READ_ONLY, new ReadOnlySecurityModule()); + bindSecurityModule(FILE, new FileBasedAccessControlModule()); + // SYSTEM: do not bind an ConnectorAccessControl so the engine will use system security with system roles + } + + private void bindSecurityModule(DeltaLakeSecurity deltaLakeSecurity, Module module) + { + install(conditionalModule( + DeltaLakeSecurityConfig.class, + security -> deltaLakeSecurity == security.getSecuritySystem(), + module)); + } +} diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/InternalDeltaLakeConnectorFactory.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/InternalDeltaLakeConnectorFactory.java index 593959e629e9..4cb172bf5cbb 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/InternalDeltaLakeConnectorFactory.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/InternalDeltaLakeConnectorFactory.java @@ -91,6 +91,7 @@ public static Connector createConnector( new CatalogNameModule(catalogName), new DeltaLakeMetastoreModule(), new DeltaLakeModule(), + new DeltaLakeSecurityModule(), binder -> { binder.bind(NodeVersion.class).toInstance(new NodeVersion(context.getNodeManager().getCurrentNode().getVersion())); binder.bind(NodeManager.class).toInstance(context.getNodeManager()); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java index 19c21f511681..2a182ad484a2 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakePlugin.java @@ -16,11 +16,16 @@ import com.google.common.collect.ImmutableMap; import io.airlift.bootstrap.ApplicationConfigurationException; import io.trino.spi.Plugin; +import io.trino.spi.connector.Connector; import io.trino.spi.connector.ConnectorFactory; import io.trino.testing.TestingConnectorContext; import org.testng.annotations.Test; +import java.io.File; +import java.nio.file.Files; + import static com.google.common.collect.Iterables.getOnlyElement; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TestDeltaLakePlugin @@ -136,4 +141,58 @@ public void testNoActiveDataFilesCaching() "delta.metadata.live-files.cache-ttl", "0s"), new TestingConnectorContext()); } + + @Test + public void testReadOnlyAllAccessControl() + { + Plugin plugin = new DeltaLakePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + + factory.create( + "test", + ImmutableMap.builder() + .put("hive.metastore.uri", "thrift://foo:1234") + .put("delta.security", "read-only") + .buildOrThrow(), + new TestingConnectorContext()) + .shutdown(); + } + + @Test + public void testSystemAccessControl() + { + Plugin plugin = new DeltaLakePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + + Connector connector = factory.create( + "test", + ImmutableMap.builder() + .put("hive.metastore.uri", "thrift://foo:1234") + .put("delta.security", "system") + .buildOrThrow(), + new TestingConnectorContext()); + assertThatThrownBy(connector::getAccessControl).isInstanceOf(UnsupportedOperationException.class); + connector.shutdown(); + } + + @Test + public void testFileBasedAccessControl() + throws Exception + { + Plugin plugin = new DeltaLakePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + File tempFile = File.createTempFile("test-delta-lake-plugin-access-control", ".json"); + tempFile.deleteOnExit(); + Files.write(tempFile.toPath(), "{}".getBytes(UTF_8)); + + factory.create( + "test", + ImmutableMap.builder() + .put("hive.metastore.uri", "thrift://foo:1234") + .put("delta.security", "file") + .put("security.config-file", tempFile.getAbsolutePath()) + .buildOrThrow(), + new TestingConnectorContext()) + .shutdown(); + } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSecurityConfig.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSecurityConfig.java new file mode 100644 index 000000000000..dde1c856ddd4 --- /dev/null +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSecurityConfig.java @@ -0,0 +1,48 @@ +/* + * 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 io.trino.plugin.deltalake; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity.ALLOW_ALL; +import static io.trino.plugin.deltalake.DeltaLakeSecurityConfig.DeltaLakeSecurity.READ_ONLY; + +public class TestDeltaLakeSecurityConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(DeltaLakeSecurityConfig.class) + .setSecuritySystem(ALLOW_ALL)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("delta.security", "read-only") + .buildOrThrow(); + + DeltaLakeSecurityConfig expected = new DeltaLakeSecurityConfig() + .setSecuritySystem(READ_ONLY); + + assertFullMapping(properties, expected); + } +}