diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java index b1cb8753d298..0ffda3677752 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/BasePgSQLTest.java @@ -22,21 +22,41 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.yb.client.IsInitDbDoneResponse; -import org.yb.client.Partition; import org.yb.client.TestUtils; import org.yb.minicluster.BaseMiniClusterTest; import org.yb.minicluster.MiniYBCluster; import org.yb.minicluster.MiniYBClusterBuilder; -import org.yb.util.*; +import org.yb.pgsql.cleaners.ClusterCleaner; +import org.yb.pgsql.cleaners.ConnectionCleaner; +import org.yb.pgsql.cleaners.UserObjectCleaner; +import org.yb.util.EnvAndSysPropertyUtil; +import org.yb.util.SanitizerUtil; import java.io.File; -import java.sql.*; -import java.util.*; +import java.net.InetSocketAddress; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; -import static org.yb.AssertionWrappers.*; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.yb.AssertionWrappers.assertArrayEquals; +import static org.yb.AssertionWrappers.assertEquals; +import static org.yb.AssertionWrappers.assertFalse; +import static org.yb.AssertionWrappers.assertTrue; +import static org.yb.AssertionWrappers.fail; +import static org.yb.util.SanitizerUtil.isASAN; import static org.yb.util.SanitizerUtil.isTSAN; public class BasePgSQLTest extends BaseMiniClusterTest { @@ -45,16 +65,13 @@ public class BasePgSQLTest extends BaseMiniClusterTest { // Postgres settings. protected static final String DEFAULT_PG_DATABASE = "postgres"; protected static final String DEFAULT_PG_USER = "postgres"; - protected static final String DEFAULT_PG_PASSWORD = ""; + public static final String TEST_PG_USER = "postgres_test"; // Postgres flags. private static final String MASTERS_FLAG = "FLAGS_pggate_master_addresses"; private static final String PG_DATA_FLAG = "PGDATA"; private static final String YB_ENABLED_IN_PG_ENV_VAR_NAME = "YB_ENABLED_IN_POSTGRES"; - // Extra Postgres flags. - protected static Map extraPostgresEnvVars; - // CQL and Redis settings. protected static boolean startCqlProxy = false; protected static boolean startRedisProxy = false; @@ -63,7 +80,8 @@ public class BasePgSQLTest extends BaseMiniClusterTest { protected File pgBinDir; - private static List connectionsToClose = new ArrayList<>(); + // Post-test cleaners, stored in a tree-map to maintain order. + private final TreeMap cleanersByPriority = getCleaners(); protected static final int DEFAULT_STATEMENT_TIMEOUT_MS = 30000; @@ -145,8 +163,9 @@ protected String pgPrefetchLimit() { protected Map getTServerFlags() { Map flagMap = new TreeMap<>(); - if (isTSAN()) { + if (isTSAN() || isASAN()) { flagMap.put("yb_client_admin_operation_timeout_sec", "120"); + flagMap.put("pggate_rpc_timeout_secs", "120"); } flagMap.put("start_cql_proxy", Boolean.toString(startCqlProxy)); flagMap.put("start_redis_proxy", Boolean.toString(startRedisProxy)); @@ -230,29 +249,44 @@ public void initPostgresBefore() throws Exception { connection.close(); connection = null; } - connection = createConnection(); + + // Create test role. + try (Connection initialConnection = newConnectionBuilder().setUser(DEFAULT_PG_USER).connect(); + Statement statement = initialConnection.createStatement()) { + statement.execute( + "CREATE ROLE " + TEST_PG_USER + " SUPERUSER CREATEROLE CREATEDB BYPASSRLS LOGIN"); + } + + connection = newConnectionBuilder().connect(); pgInitialized = true; } - private void configureDefaultConnectionOptions(Connection conn) throws Exception { - conn.setTransactionIsolation(IsolationLevel.DEFAULT.pgIsolationLevel); - conn.setAutoCommit(AutoCommit.DEFAULT.enabled); + static ConnectionBuilder newConnectionBuilder() { + return new ConnectionBuilder(miniCluster); } + /** + * @deprecated Use {@link #newConnectionBuilder()} instead. + */ + @Deprecated protected Connection createConnection( IsolationLevel isolationLevel, AutoCommit autoCommit) throws Exception { - Connection conn = createConnection(); - conn.setTransactionIsolation(isolationLevel.pgIsolationLevel); - conn.setAutoCommit(autoCommit.enabled); - return conn; + return newConnectionBuilder() + .setIsolationLevel(isolationLevel) + .setAutoCommit(autoCommit) + .connect(); } + /** + * @deprecated Use {@link #newConnectionBuilder()} instead. + */ + @Deprecated protected Connection createConnectionSerializableNoAutoCommit() throws Exception { - return createConnection( - IsolationLevel.SERIALIZABLE, - AutoCommit.DISABLED - ); + return newConnectionBuilder() + .setIsolationLevel(IsolationLevel.SERIALIZABLE) + .setAutoCommit(AutoCommit.DISABLED) + .connect(); } public String getPgHost(int tserverIndex) { @@ -263,77 +297,48 @@ public int getPgPort(int tserverIndex) { return miniCluster.getPostgresContactPoints().get(tserverIndex).getPort(); } + /** + * @deprecated Use {@link #newConnectionBuilder()} instead. + */ + @Deprecated protected Connection createConnection() throws Exception { - return createConnection(0); + return newConnectionBuilder().connect(); } + /** + * @deprecated Use {@link #newConnectionBuilder()} instead. + */ + @Deprecated protected Connection createConnection(int tserverIndex) throws Exception { - return createConnection(tserverIndex, DEFAULT_PG_DATABASE); + return newConnectionBuilder() + .setTServer(tserverIndex) + .connect(); } + /** + * @deprecated Use {@link #newConnectionBuilder()} instead. + */ + @Deprecated protected Connection createPgConnectionToTServer( int tserverIndex, IsolationLevel isolationLevel, AutoCommit autoCommit) throws Exception { - Connection conn = createConnection(tserverIndex, DEFAULT_PG_DATABASE); - conn.setTransactionIsolation(isolationLevel.pgIsolationLevel); - conn.setAutoCommit(autoCommit.enabled); - return conn; + return newConnectionBuilder() + .setTServer(tserverIndex) + .setIsolationLevel(isolationLevel) + .setAutoCommit(autoCommit) + .connect(); } + /** + * @deprecated Use {@link #newConnectionBuilder()} instead. + */ + @Deprecated protected Connection createConnection(int tserverIndex, String pgDB) throws Exception { - final String pgHost = getPgHost(tserverIndex); - final int pgPort = getPgPort(tserverIndex); - String url = String.format("jdbc:postgresql://%s:%d/%s", pgHost, pgPort, pgDB); - if (EnvAndSysPropertyUtil.isEnvVarOrSystemPropertyTrue("YB_PG_JDBC_TRACE_LOGGING")) { - url += "?loggerLevel=TRACE"; - } - - final int MAX_ATTEMPTS = 10; - int delayMs = 500; - Connection connection = null; - for (int attempt = 1; attempt <= MAX_ATTEMPTS; ++attempt) { - try { - connection = DriverManager.getConnection(url, DEFAULT_PG_USER, DEFAULT_PG_PASSWORD); - if (connection == null) { - throw new NullPointerException("getConnection returned null"); - } - connectionsToClose.add(connection); - configureDefaultConnectionOptions(connection); - // JDBC does not specify a default for auto-commit, let's set it to true here for - // determinism. - connection.setAutoCommit(true); - return connection; - } catch (SQLException sqlEx) { - // Close the connection now if we opened it, instead of waiting until the end of the test. - if (connection != null) { - try { - connection.close(); - connectionsToClose.remove(connection); - connection = null; - } catch (SQLException closingError) { - LOG.error("Failure to close connection during failure cleanup before a retry:", - closingError); - LOG.error("When handling this exception when opening/setting up connection:", sqlEx); - } - } - - if (attempt < MAX_ATTEMPTS && - sqlEx.getMessage().contains("FATAL: the database system is starting up") || - sqlEx.getMessage().contains("refused. Check that the hostname and port are correct " + - "and that the postmaster is accepting")) { - LOG.info("Postgres is still starting up, waiting for " + delayMs + " ms. " + - "Got message: " + sqlEx.getMessage()); - Thread.sleep(delayMs); - delayMs = Math.min(delayMs + 500, 10000); - continue; - } - LOG.error("Exception while trying to create connection (after " + attempt + - " attempts): " + sqlEx.getMessage()); - throw sqlEx; - } - } - throw new IllegalStateException("Should not be able to reach here"); + return newConnectionBuilder() + .setTServer(tserverIndex) + .setDatabase(pgDB) + .connect(); } protected Map getPgRegressEnvVars() { @@ -357,79 +362,46 @@ protected Map getPgRegressEnvVars() { return pgRegressEnvVars; } + /** + * Register default post-test cleaners. + */ + protected TreeMap getCleaners() { + TreeMap cleaners = new TreeMap<>(); + cleaners.put(99, new UserObjectCleaner()); + cleaners.put(100, new ConnectionCleaner()); + return cleaners; + } + @After public void cleanUpAfter() throws Exception { if (connection == null) { - LOG.warn("No connection created, skipping dropping tables"); + LOG.warn("No connection created, skipping cleanup"); return; } - try (Statement statement = connection.createStatement()) { - DatabaseMetaData dbmd = connection.getMetaData(); - String[] views = {"VIEW"}; - ResultSet rs = dbmd.getTables(null, null, "%", views); - while (rs.next()) { - statement.execute("DROP VIEW " + rs.getString("TABLE_NAME") + " CASCADE"); - } - String[] tables = {"TABLE"}; - rs = dbmd.getTables(null, null, "%", tables); - while (rs.next()) { - statement.execute("DROP TABLE " + rs.getString("TABLE_NAME") + " CASCADE"); - } + + // If root connection was closed, open a new one for cleaning. + if (connection.isClosed()) { + connection = newConnectionBuilder().connect(); + } + + // Run cleaners in ascending key order (i.e. low key => high priority). + for (Map.Entry entry : cleanersByPriority.entrySet()) { + entry.getValue().clean(connection); } } @AfterClass public static void tearDownAfter() throws Exception { - try { - tearDownPostgreSQL(); - } finally { - LOG.info("Destroying mini-cluster"); - if (miniCluster != null) { - destroyMiniCluster(); - miniCluster = null; - } + // Close the root connection, which is not cleaned up after each test. + if (connection != null && !connection.isClosed()) { + connection.close(); } - } - - private static void tearDownPostgreSQL() throws Exception { - if (connection != null) { - try (Statement statement = connection.createStatement()) { - try (ResultSet resultSet = statement.executeQuery( - "SELECT client_hostname, client_port, state, query, pid FROM pg_stat_activity")) { - while (resultSet.next()) { - int backendPid = resultSet.getInt(5); - LOG.info("Found connection: " + - "hostname=" + resultSet.getString(1) + ", " + - "port=" + resultSet.getInt(2) + ", " + - "state=" + resultSet.getString(3) + ", " + - "query=" + resultSet.getString(4) + ", " + - "backend_pid=" + backendPid); - } - } - } - catch (SQLException e) { - LOG.info("Exception when trying to list PostgreSQL connections", e); - } - - LOG.info("Closing connections."); - for (Connection connection : connectionsToClose) { - try { - if (connection == null) { - LOG.error("connectionsToClose contains a null connection!"); - } else { - connection.close(); - } - } catch (SQLException ex) { - LOG.error("Exception while trying to close connection"); - throw ex; - } - } - } else { - LOG.info("Connection is already null, nothing to close"); + pgInitialized = false; + LOG.info("Destroying mini-cluster"); + if (miniCluster != null) { + destroyMiniCluster(); + miniCluster = null; } - LOG.info("Finished closing connection."); - - LOG.info("Finished stopping postgres server."); } /** @@ -920,4 +892,129 @@ protected void timeQueryWithRowCount(String stmt, int expectedRowCount, long max LOG.info(String.format("Complete query: %s. Elapsed time = %d msecs", stmt, elapsedTimeMillis)); assertTrue(elapsedTimeMillis < maxRuntimeMillis); } + + public static class ConnectionBuilder { + private static final int MAX_CONNECTION_ATTEMPTS = 10; + private static final int INITIAL_CONNECTION_DELAY_MS = 500; + + private final MiniYBCluster miniCluster; + + private int tserverIndex = 0; + private String database = DEFAULT_PG_DATABASE; + private String user = TEST_PG_USER; + private String password = null; + private IsolationLevel isolationLevel = IsolationLevel.DEFAULT; + private AutoCommit autoCommit = AutoCommit.DEFAULT; + + ConnectionBuilder(MiniYBCluster miniCluster) { + this.miniCluster = miniCluster; + } + + ConnectionBuilder setTServer(int tserverIndex) { + this.tserverIndex = tserverIndex; + return this; + } + + ConnectionBuilder setDatabase(String database) { + this.database = database; + return this; + } + + ConnectionBuilder setUser(String user) { + this.user = user; + return this; + } + + ConnectionBuilder setPassword(String password) { + this.password = password; + return this; + } + + ConnectionBuilder setIsolationLevel(IsolationLevel isolationLevel) { + this.isolationLevel = isolationLevel; + return this; + } + + ConnectionBuilder setAutoCommit(AutoCommit autoCommit) { + this.autoCommit = autoCommit; + return this; + } + + ConnectionBuilder newBuilder() { + return new ConnectionBuilder(miniCluster) + .setTServer(tserverIndex) + .setDatabase(database) + .setUser(user) + .setPassword(password) + .setIsolationLevel(isolationLevel) + .setAutoCommit(autoCommit); + } + + Connection connect() throws Exception { + final InetSocketAddress postgresAddress = miniCluster.getPostgresContactPoints() + .get(tserverIndex); + String url = String.format( + "jdbc:postgresql://%s:%d/%s", + postgresAddress.getHostName(), + postgresAddress.getPort(), + database + ); + if (EnvAndSysPropertyUtil.isEnvVarOrSystemPropertyTrue("YB_PG_JDBC_TRACE_LOGGING")) { + url += "?loggerLevel=TRACE"; + } + + int delayMs = INITIAL_CONNECTION_DELAY_MS; + for (int attempt = 1; attempt <= MAX_CONNECTION_ATTEMPTS; ++attempt) { + Connection connection = null; + try { + connection = checkNotNull(DriverManager.getConnection(url, user, password)); + + connection.setTransactionIsolation(isolationLevel.pgIsolationLevel); + connection.setAutoCommit(autoCommit.enabled); + + ConnectionCleaner.register(connection); + return connection; + } catch (SQLException sqlEx) { + // Close the connection now if we opened it, instead of waiting until the end of the test. + if (connection != null) { + try { + connection.close(); + } catch (SQLException closingError) { + LOG.error("Failure to close connection during failure cleanup before a retry:", + closingError); + LOG.error("When handling this exception when opening/setting up connection:", sqlEx); + } + } + + boolean retry = false; + + if (attempt < MAX_CONNECTION_ATTEMPTS) { + if (sqlEx.getMessage().contains("FATAL: the database system is starting up") + || sqlEx.getMessage().contains("refused. Check that the hostname and port are " + + "correct and that the postmaster is accepting")) { + retry = true; + + LOG.info("Postgres is still starting up, waiting for " + delayMs + " ms. " + + "Got message: " + sqlEx.getMessage()); + } else if (sqlEx.getMessage().contains("the database system is in recovery mode")) { + retry = true; + + LOG.info("Postgres is in recovery mode, waiting for " + delayMs + " ms. " + + "Got message: " + sqlEx.getMessage()); + } + } + + if (retry) { + Thread.sleep(delayMs); + delayMs = Math.min(delayMs + 500, 10000); + } else { + LOG.error("Exception while trying to create connection (after " + attempt + + " attempts): " + sqlEx.getMessage()); + throw sqlEx; + } + } + } + throw new IllegalStateException("Should not be able to reach here"); + } + } } diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAuthorization.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAuthorization.java new file mode 100644 index 000000000000..5d6ba1000b32 --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgAuthorization.java @@ -0,0 +1,3271 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql; + +import com.google.common.collect.Lists; +import org.hamcrest.CoreMatchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yb.pgsql.cleaners.ClusterCleaner; +import org.yb.pgsql.cleaners.DatabaseCleaner; +import org.yb.pgsql.cleaners.RoleCleaner; +import org.yb.util.YBTestRunnerNonTsanOnly; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.Calendar; +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import static org.yb.AssertionWrappers.assertEquals; +import static org.yb.AssertionWrappers.assertFalse; +import static org.yb.AssertionWrappers.assertNotEquals; +import static org.yb.AssertionWrappers.assertThat; +import static org.yb.AssertionWrappers.assertTrue; +import static org.yb.AssertionWrappers.fail; + +/** + * Tests for PostgreSQL RBAC. + */ +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestPgAuthorization extends BasePgSQLTest { + private static final Logger LOG = LoggerFactory.getLogger(TestPgAuthorization.class); + + private static final String PERMISSION_DENIED = "permission denied"; + + // Host-based authentication config which requires a password for the pass_role user. + private static final String CUSTOM_PG_HBA_CONFIG = "" + + "host all pass_role 0.0.0.0/0 password\n" + + "host all pass_role ::0/0 password\n" + + "host all all 0.0.0.0/0 trust\n" + + "host all all ::0/0 trust\n"; + + @Override + protected Map getTServerFlags() { + Map flags = super.getTServerFlags(); + + try { + flags.put("pgsql_hba_conf_file", String.format("%s", createPgHBAConf().getName())); + } catch (IOException ioe) { + throw new RuntimeException("Failed to write custom pg_hba.conf", ioe); + } + + return flags; + } + + private static File createPgHBAConf() throws IOException { + File hbaFile = File.createTempFile( + "yb_pg_hba_conf_", + "", + new File(System.getProperty("user.dir")) + ); + hbaFile.deleteOnExit(); + + try (BufferedWriter writer = new BufferedWriter(new FileWriter(hbaFile))) { + writer.write(CUSTOM_PG_HBA_CONFIG); + } + + return hbaFile; + } + + @Override + protected TreeMap getCleaners() { + TreeMap cleaners = super.getCleaners(); + cleaners.put(0, new DatabaseCleaner()); + cleaners.put(1, new RoleCleaner()); + return cleaners; + } + + @Test + public void testDefaultAuthorization() throws Exception { + try (Statement statement = connection.createStatement()) { + // Default users are correctly set. + assertEquals(TEST_PG_USER, getSessionUser(statement)); + assertEquals(TEST_PG_USER, getCurrentUser(statement)); + } + } + + @Test + public void testSessionAuthorization() throws Exception { + try (Statement statement = connection.createStatement()) { + // Create some roles. + statement.execute("CREATE ROLE unprivileged"); + statement.execute("CREATE ROLE su LOGIN SUPERUSER"); + statement.execute("CREATE ROLE some_role LOGIN"); + statement.execute("CREATE ROLE some_group ROLE some_role"); + } + + try (Connection connection = newConnectionBuilder().setUser("su").connect(); + Statement statement = connection.createStatement()) { + assertEquals("su", getSessionUser(statement)); + assertEquals("su", getCurrentUser(statement)); + + // Superuser can set session authorization. + statement.execute("SET SESSION AUTHORIZATION unprivileged"); + + assertEquals("unprivileged", getSessionUser(statement)); + assertEquals("unprivileged", getCurrentUser(statement)); + + // Session authorization determines membership. + runInvalidQuery(statement, "SET ROLE some_role", PERMISSION_DENIED); + + // Original session user is used when determining privileges. + statement.execute("SET SESSION AUTHORIZATION su"); + + assertEquals("su", getSessionUser(statement)); + assertEquals("su", getCurrentUser(statement)); + + statement.execute("SET SESSION AUTHORIZATION unprivileged"); + statement.execute("SET SESSION AUTHORIZATION some_role"); + statement.execute("SET ROLE some_group"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_group", getCurrentUser(statement)); + + // Session authorization can be reset, and it resets the current role. + statement.execute("RESET SESSION AUTHORIZATION"); + + assertEquals("su", getSessionUser(statement)); + assertEquals("su", getCurrentUser(statement)); + + // Using DEFAULT is equivalent to RESET. + statement.execute("SET SESSION AUTHORIZATION unprivileged"); + statement.execute("SET SESSION AUTHORIZATION DEFAULT"); + + assertEquals("su", getSessionUser(statement)); + assertEquals("su", getCurrentUser(statement)); + + // SET LOCAL is only valid for current transaction. + statement.execute("BEGIN"); + statement.execute("SET LOCAL SESSION AUTHORIZATION unprivileged"); + + assertEquals("unprivileged", getSessionUser(statement)); + assertEquals("unprivileged", getCurrentUser(statement)); + + statement.execute("COMMIT"); + + assertEquals("su", getSessionUser(statement)); + assertEquals("su", getCurrentUser(statement)); + + // SET SESSION is valid for the entire session. + statement.execute("BEGIN"); + statement.execute("SET SESSION SESSION AUTHORIZATION unprivileged"); + statement.execute("COMMIT"); + + assertEquals("unprivileged", getSessionUser(statement)); + assertEquals("unprivileged", getCurrentUser(statement)); + } + + try (Connection connection = newConnectionBuilder().setUser("su").connect(); + Statement statement = connection.createStatement()) { + // Users have been reset, since this is a new session. + assertEquals("su", getSessionUser(statement)); + assertEquals("su", getCurrentUser(statement)); + } + + try (Connection connection = newConnectionBuilder().setUser("some_role").connect(); + Statement statement = connection.createStatement()) { + statement.execute("SET ROLE some_group"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_group", getCurrentUser(statement)); + + // Non-superuser can set session authorization to the role they authenticated as. + statement.execute("SET SESSION AUTHORIZATION some_role"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + // Non-superuser cannot set session authorization to other roles. + runInvalidQuery(statement, "SET SESSION AUTHORIZATION some_group", PERMISSION_DENIED); + runInvalidQuery(statement, "SET SESSION AUTHORIZATION unprivileged", PERMISSION_DENIED); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + statement.execute("SET ROLE some_group"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_group", getCurrentUser(statement)); + + // Non-superuser can reset their session authorization. + statement.execute("RESET SESSION AUTHORIZATION"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + } + + final AtomicInteger state = new AtomicInteger(0); + final Lock lock = new ReentrantLock(); + final Condition condition = lock.newCondition(); + + Thread thread = new Thread(() -> { + try (Connection connection = newConnectionBuilder().setUser("su").connect(); + Statement statement = connection.createStatement()) { + // Signal that superuser session has started. + lock.lock(); + state.incrementAndGet(); + condition.signal(); + + // Wait for superuser attribute to be revoked. + while (state.get() != 2) { + condition.await(); + } + lock.unlock(); + + // Can still set session authorization, even though "su" is no longer a superuser. + statement.execute("SET SESSION AUTHORIZATION unprivileged"); + assertEquals("unprivileged", getSessionUser(statement)); + assertEquals("unprivileged", getCurrentUser(statement)); + } catch (Exception e) { + throw new RuntimeException(e); + } + + try (Connection connection = newConnectionBuilder().setUser("su").connect(); + Statement statement = connection.createStatement()) { + // In a new session, "su" can no longer set session authorization. + runInvalidQuery(statement, "SET SESSION AUTHORIZATION unprivileged", PERMISSION_DENIED); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + thread.start(); + + // Wait for superuser session to start. + lock.lock(); + while (state.get() != 1) { + condition.await(); + } + lock.unlock(); + + try (Statement statement = connection.createStatement()) { + // Revoke superuser from "su". + statement.execute("ALTER ROLE su NOSUPERUSER"); + } + + // Signal that superuser attribute has been revoked. + lock.lock(); + state.incrementAndGet(); + condition.signal(); + lock.unlock(); + + thread.join(); + } + + @Test + public void testRoleChanging() throws Exception { + try (Statement statement = connection.createStatement()) { + // Create some roles. + statement.execute("CREATE ROLE su LOGIN SUPERUSER"); + statement.execute("CREATE ROLE unprivileged"); + statement.execute("CREATE ROLE some_role LOGIN"); + statement.execute("CREATE ROLE some_group LOGIN ROLE some_role"); + statement.execute("CREATE ROLE some_large_group ROLE some_group"); + statement.execute("CREATE ROLE other_group ROLE some_role"); + } + + try (Connection connection = newConnectionBuilder().setUser("some_role").connect(); + Statement statement = connection.createStatement()) { + // Cannot set roles to those who we aren't a member of. + runInvalidQuery(statement, "SET ROLE unprivileged", PERMISSION_DENIED); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + // Can set role to current role. + statement.execute("SET ROLE some_role"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + // Can set role to parent role. + statement.execute("SET ROLE some_group"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_group", getCurrentUser(statement)); + + // Can reset role. + statement.execute("RESET ROLE"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + // NONE is equivalent to RESET. + statement.execute("SET ROLE some_group"); + statement.execute("SET ROLE NONE"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + // Can set role to any role in inheritance tree. + statement.execute("SET ROLE some_large_group"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_large_group", getCurrentUser(statement)); + + // SET privileges depend on session user, not current user. + statement.execute("SET ROLE other_group"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("other_group", getCurrentUser(statement)); + + // SET LOCAL is only valid for current transaction. + statement.execute("BEGIN"); + statement.execute("SET LOCAL ROLE some_role"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + + statement.execute("COMMIT"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("other_group", getCurrentUser(statement)); + + // SET SESSION is valid for entire session. + statement.execute("BEGIN"); + statement.execute("SET SESSION ROLE some_group"); + statement.execute("COMMIT"); + + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_group", getCurrentUser(statement)); + } + + try (Connection connection = newConnectionBuilder().setUser("some_role").connect(); + Statement statement = connection.createStatement()) { + // Users have been reset, since this is a new session. + assertEquals("some_role", getSessionUser(statement)); + assertEquals("some_role", getCurrentUser(statement)); + } + + try (Connection connection = newConnectionBuilder().setUser("some_group").connect(); + Statement statement = connection.createStatement()) { + // Cannot set roles to those who are a members of the current role. + runInvalidQuery(statement, "SET ROLE some_role", PERMISSION_DENIED); + } + + try (Connection connection = newConnectionBuilder().setUser("su").connect(); + Statement statement = connection.createStatement()) { + // Superuser can set any role. + statement.execute("SET ROLE unprivileged"); + statement.execute("SET ROLE some_role"); + statement.execute("SET ROLE some_large_group"); + } + } + + @Test + public void testAttributes() throws Exception { + // NOTE: The INHERIT attribute is tested separately in testMembershipInheritance. + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE unprivileged"); + + /* + * SUPERUSER + */ + + // Superuser can create other superusers. + statement.execute("CREATE ROLE su SUPERUSER"); + + // Superuser can alter superuser roles. + statement.execute("ALTER ROLE su LOGIN"); + statement.execute("ALTER ROLE su NOLOGIN"); + + // Superusers can create and drop databases. + statement.execute("CREATE DATABASE test_su_db"); + statement.execute("DROP DATABASE test_su_db"); + + // Superusers can change superuser settings. + statement.execute("SET deadlock_timeout=2000"); + statement.execute("RESET deadlock_timeout"); + + // Superusers cannot login without LOGIN. + try (Connection ignored = newConnectionBuilder().setUser("su").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("role \"su\" is not permitted to log in") + ); + } + + statement.execute("CREATE ROLE su_login SUPERUSER LOGIN"); + try (Connection suConnection = newConnectionBuilder().setUser("su_login").connect(); + Statement suStatement = suConnection.createStatement()) { + suStatement.execute("CREATE TABLE su_login_table(id int)"); + } + + // Superusers bypass ownership and permissions checks. + statement.execute("DROP TABLE su_login_table"); + + // Superusers can read/write/execute server files. + statement.execute("CREATE TABLE files(filename text)"); + statement.execute("SELECT pg_read_file('/non/existent/file', 0, 0, true)"); + statement.execute("COPY (SELECT * FROM pg_roles) TO '/dev/null'"); + statement.execute("COPY files FROM PROGRAM 'ls /usr/bin'"); + + /* + * CREATEDB + */ + + statement.execute("CREATE ROLE cdb_role CREATEDB"); + + withRole(statement, "cdb_role", () -> { + // User with CREATEDB can create a database. + statement.execute("CREATE DATABASE cdb_role_db"); + statement.execute("DROP DATABASE cdb_role_db"); + }); + + withRole(statement, "unprivileged", () -> { + // Unprivileged users cannot create databases. + runInvalidQuery(statement, "CREATE DATABASE cdb_role_db", PERMISSION_DENIED); + }); + + /* + * CREATEROLE + */ + + statement.execute("CREATE ROLE cr_role CREATEROLE"); + + withRole(statement, "cr_role", () -> { + // User with CREATEROLE can create a role. + statement.execute("CREATE ROLE cr_created_role"); + statement.execute("DROP ROLE cr_created_role"); + + // Non-superusers cannot create superuser roles. + runInvalidQuery( + statement, + "CREATE ROLE cr_created_role SUPERUSER", + "must be superuser to create superusers" + ); + + // Non-superusers cannot alter superuser roles. + runInvalidQuery( + statement, + "ALTER ROLE su LOGIN", + "must be superuser to alter superusers" + ); + }); + + withRole(statement, "unprivileged", () -> { + // Unprivileged users cannot create roles. + runInvalidQuery(statement, "CREATE ROLE cr_created_role", PERMISSION_DENIED); + }); + + /* + * LOGIN + */ + + statement.execute("CREATE ROLE login_user LOGIN"); + + // Users with LOGIN can connect directly. + try (Connection ignored = newConnectionBuilder().setUser("login_user").connect()) { + // No-op. + } + + // Users without LOGIN cannot connect directly. + try (Connection ignored = newConnectionBuilder().setUser("unprivileged").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("role \"unprivileged\" is not permitted to log in") + ); + } + + /* + * REPLICATION + */ + + // Replication is not currently supported, but we should still be able to add the attribute. + statement.execute("CREATE ROLE replication_user REPLICATION"); + + /* + * BYPASSRLS + */ + + // RLS is not currently supported, but we should still be able to add the attribute. + statement.execute("CREATE ROLE bypassrls_user BYPASSRLS"); + + /* + * CONNECTION LIMIT + */ + + statement.execute("CREATE ROLE limit_role LOGIN CONNECTION LIMIT 2"); + + try (Connection ignored1 = newConnectionBuilder().setUser("limit_role").connect()) { + try (Connection connection2 = newConnectionBuilder().setUser("limit_role").connect()) { + // Third concurrent connection causes error. + try (Connection ignored3 = newConnectionBuilder().setUser("limit_role").connect()) { + fail("Expected third login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("too many connections for role \"limit_role\"") + ); + } + + // Close second connection. + connection2.close(); + + // New connection now succeeds. + try (Connection ignored2 = newConnectionBuilder().setUser("limit_role").connect()) { + // No-op. + } + } + } + + /* + * PASSWORD + */ + + // Create role with password. + statement.execute("DROP ROLE IF EXISTS pass_role"); + statement.execute("CREATE ROLE pass_role LOGIN PASSWORD 'pass1'"); + + // Password is encrypted, despite not being specified. + ResultSet password_result = statement.executeQuery( + "SELECT rolpassword FROM pg_authid WHERE rolname='pass_role'"); + password_result.next(); + String password_hash = password_result.getString(1); + assertNotEquals(password_hash, ""); + assertNotEquals(password_hash, "pass1"); + + // Can login with password. + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("pass1").connect()) { + // No-op. + } + + // Cannot login without password. + try (Connection ignored = newConnectionBuilder().setUser("pass_role").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("no password was provided") + ); + } + + // Cannot login with incorrect password. + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("wrong").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("password authentication failed for user \"pass_role\"") + ); + } + + // Password does not imply login. + statement.execute("DROP ROLE IF EXISTS pass_role"); + statement.execute("CREATE ROLE pass_role PASSWORD 'pass1'"); + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("pass1").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("role \"pass_role\" is not permitted to log in") + ); + } + + /* + * ENCRYPTED PASSWORD + */ + + // Create role with encrypted password. + statement.execute("DROP ROLE IF EXISTS pass_role"); + statement.execute("CREATE ROLE pass_role LOGIN ENCRYPTED PASSWORD 'pass2'"); + + // Password is encrypted. + password_result = statement.executeQuery( + "SELECT rolpassword FROM pg_authid WHERE rolname='pass_role'"); + password_result.next(); + password_hash = password_result.getString(1); + assertNotEquals(password_hash, ""); + assertNotEquals(password_hash, "pass2"); + + // Can login with password. + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("pass2").connect()) { + // No-op. + } + + // Cannot login without password. + try (Connection ignored = newConnectionBuilder().setUser("pass_role").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("no password was provided") + ); + } + + // Cannot login with incorrect password. + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("wrong").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("password authentication failed for user \"pass_role\"") + ); + } + + /* + * UNENCRYPTED PASSWORD + */ + + // Cannot create users with unencrypted passwords. + statement.execute("DROP ROLE IF EXISTS pass_role"); + runInvalidQuery( + statement, + "CREATE ROLE pass_role LOGIN UNENCRYPTED PASSWORD 'pass'", + "UNENCRYPTED PASSWORD is no longer supported" + ); + + /* + * VALID UNTIL + */ + + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date()); + cal.add(Calendar.SECOND, 10); + Timestamp expiriationTime = new Timestamp(cal.getTime().getTime()); + + statement.execute("DROP ROLE IF EXISTS pass_role"); + statement.execute("CREATE ROLE pass_role LOGIN PASSWORD 'password' " + + "VALID UNTIL '" + expiriationTime.toString() + "'"); + + // Can connect now. + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("password").connect()) { + // No-op. + } + + // Wait until after the expiration time. + Thread.sleep(11000); + + // Password is no longer valid. + try (Connection ignored = newConnectionBuilder().setUser("pass_role") + .setPassword("password").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("password authentication failed for user \"pass_role\"") + ); + } + + /* + * IN ROLE + */ + + // Create group. + statement.execute("CREATE ROLE group_role NOLOGIN"); + + statement.execute("CREATE ROLE in_role_user IN ROLE group_role"); + + withRole(statement, "in_role_user", () -> { + // in_role_user role is immediately a member of group_role. + statement.execute("SET ROLE group_role"); + }); + + /* + * IN GROUP + */ + + statement.execute("CREATE ROLE in_group_user IN GROUP group_role"); + + withRole(statement, "in_group_user", () -> { + // in_group_user role is immediately a member of group_role. + statement.execute("SET ROLE group_role"); + }); + + /* + * ROLE + */ + + // Create a roles to add to other roles. + statement.execute("CREATE ROLE role_to_add"); + statement.execute("CREATE ROLE other_role_to_add"); + + statement.execute("CREATE ROLE add_role_role ROLE role_to_add"); + + withRole(statement, "role_to_add", () -> { + // role_to_add has been automatically added to the group. + statement.execute("SET ROLE add_role_role"); + statement.execute("RESET ROLE"); + + // role_to_add cannot add new members to the group. + runInvalidQuery( + statement, + "GRANT add_role_role TO other_role_to_add", + "must have admin option on role \"add_role_role\"" + ); + }); + + /* + * USER + */ + + statement.execute("CREATE ROLE add_user_role USER role_to_add"); + + withRole(statement, "role_to_add", () -> { + // role_to_add has been automatically added to the group. + statement.execute("SET ROLE add_user_role"); + statement.execute("RESET ROLE"); + + // role_to_add cannot add new members to the group. + runInvalidQuery( + statement, + "GRANT add_user_role TO other_role_to_add", + "must have admin option on role \"add_user_role\"" + ); + }); + + /* + * ADMIN + */ + + statement.execute("CREATE ROLE add_admin_role ADMIN role_to_add"); + + withRole(statement, "role_to_add", () -> { + // role_to_add has been automatically added to the group. + statement.execute("SET ROLE add_admin_role"); + statement.execute("RESET ROLE"); + + // role_to_add can add new members to the group. + statement.execute("GRANT add_admin_role to other_role_to_add"); + }); + + /* + * SYSID + */ + + // This attribute is ignored, but we should still be able to add it. + statement.execute("CREATE ROLE sysid_role SYSID 12345"); + } + } + + @Test + public void testAlterAttributes() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE can_login LOGIN"); + statement.execute("CREATE ROLE cannot_login"); + + // Alter roles to switch attributes. + statement.execute("ALTER ROLE can_login NOLOGIN"); + statement.execute("ALTER ROLE cannot_login LOGIN"); + + // Role "can_login" can no longer login. + try (Connection ignored = newConnectionBuilder().setUser("can_login").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("role \"can_login\" is not permitted to log in") + ); + } + + // Role "cannot_login" can now login. + try (Connection ignored = newConnectionBuilder().setUser("cannot_login").connect()) { + // No-op. + } + } + } + + @Test + public void testSingleLevelMembershipInheritance() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE inh_role LOGIN INHERIT"); + statement.execute("CREATE ROLE no_inh_role LOGIN NOINHERIT"); + + statement.execute("CREATE ROLE create_role_group CREATEROLE ROLE inh_role, no_inh_role"); + + statement.execute("CREATE ROLE role_with_config ROLE inh_role, no_inh_role"); + statement.execute("ALTER ROLE role_with_config SET search_path='some path'"); + + statement.execute("CREATE ROLE role_with_privileges ROLE inh_role, no_inh_role"); + statement.execute("CREATE TABLE test_table(id int)"); + statement.execute("GRANT SELECT ON TABLE test_table TO role_with_privileges"); + } + + try (Connection connection = newConnectionBuilder().setUser("no_inh_role").connect(); + Statement statement = connection.createStatement()) { + // NOINHERIT user does not inherit attributes. + runInvalidQuery(statement, "CREATE ROLE test", PERMISSION_DENIED); + + // NOINHERIT user does not inherit config variables. + assertQuery(statement, "SHOW search_path", new Row("\"$user\", public")); + + // NOINHERIT user does not inherit privileges. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + } + + try (Connection connection = newConnectionBuilder().setUser("inh_role").connect(); + Statement statement = connection.createStatement()) { + // INHERIT user does not inherit attributes. + runInvalidQuery(statement, "CREATE ROLE test", PERMISSION_DENIED); + + // INHERIT user does not inherit config variables. + assertQuery(statement, "SHOW search_path", new Row("\"$user\", public")); + + // INHERIT user does inherit privileges. + assertQuery(statement, "SELECT * FROM test_table"); + } + } + + @Test + public void testMultiLevelMembershipInheritance() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE inh_role LOGIN INHERIT"); + + statement.execute("CREATE ROLE inh_group INHERIT ROLE inh_role"); + statement.execute("CREATE ROLE no_inh_group NOINHERIT ROLE inh_role"); + + statement.execute("CREATE ROLE role_with_privileges1 ROLE inh_group"); + statement.execute("CREATE TABLE test_table1(id int)"); + statement.execute("GRANT SELECT ON TABLE test_table1 TO role_with_privileges1"); + + statement.execute("CREATE ROLE role_with_privileges2 ROLE no_inh_group"); + statement.execute("CREATE TABLE test_table2(id int)"); + statement.execute("GRANT SELECT ON TABLE test_table2 TO role_with_privileges2"); + + withRole(statement, "inh_role", () -> { + // Base role inherits privileges from parent of INHERIT group. + assertQuery(statement, "SELECT * FROM test_table1"); + + // Base role does not inherit privileges from parent of NOINHERIT group. + runInvalidQuery(statement, "SELECT * FROM test_table2", PERMISSION_DENIED); + }); + } + } + + @Test + public void testDefaultRoles() throws Exception { + RoleSet roles = new RoleSet("unprivileged", "pg_read_all_settings", "pg_read_all_stats", + "pg_stat_scan_tables", "pg_signal_backend", "pg_read_server_files", "pg_write_server_files", + "pg_execute_server_program", "pg_monitor"); + + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE unprivileged"); + + /* + * pg_read_all_settings + */ + + withRole(statement, "pg_read_all_settings", () -> { + // Can read superuser-only settings. + assertTrue(statement.executeQuery( + "SELECT * FROM pg_settings WHERE name='dynamic_library_path'").next()); + + // Cannot change superuser settings. + runInvalidQuery(statement, "SET dynamic_library_path='/somedir'", PERMISSION_DENIED); + runInvalidQuery(statement, "RESET dynamic_library_path", PERMISSION_DENIED); + }); + + withRoles(statement, roles.excluding("pg_read_all_settings", "pg_monitor"), () -> { + // Other roles cannot read superuser-only settings. + assertFalse(statement.executeQuery( + "SELECT * FROM pg_settings WHERE name='dynamic_library_path'").next()); + + // Other roles cannot change superuser settings. + runInvalidQuery(statement, "SET dynamic_library_path='/somedir'", PERMISSION_DENIED); + runInvalidQuery(statement, "RESET dynamic_library_path", PERMISSION_DENIED); + }); + + /* + * pg_read_all_stats + */ + + withRole(statement, "pg_read_all_stats", () -> { + // Can read all statistics views without censorship. + statement.execute("SELECT query FROM pg_stat_activity"); + ResultSet resultSet = statement.getResultSet(); + while (resultSet.next()) { + assertNotEquals(resultSet.getString(1), ""); + } + }); + + // Other roles (except pg_monitor) have some statistics censored. + withRoles(statement, roles.excluding("pg_read_all_stats", "pg_monitor"), () -> { + statement.execute("SELECT query FROM pg_stat_activity"); + ResultSet results = statement.getResultSet(); + while (results.next()) { + assertEquals(results.getString(1), ""); + } + }); + + /* + * pg_stat_scan_tables + */ + + // I am unaware of any monitoring functions in vanilla postgres which require the + // pg_stat_scan_tables role, so just test that it exists. + statement.execute("SET SESSION AUTHORIZATION pg_stat_scan_tables"); + statement.execute("RESET SESSION AUTHORIZATION"); + + /* + * pg_signal_backend + */ + + // Create a new non-superuser user, whose queries we can kill. + statement.execute("CREATE ROLE test_user LOGIN;"); + + final AtomicInteger testBackendPid = new AtomicInteger(-1); + final Lock lock = new ReentrantLock(); + final Condition condition = lock.newCondition(); + + new Thread(() -> { + try (Connection testConnection = newConnectionBuilder().setUser("test_user").connect(); + Statement testStatement = testConnection.createStatement()) { + // Get pid of backend process for this connection. + ResultSet resultSet = testStatement.executeQuery("SELECT pg_backend_pid()"); + resultSet.next(); + + lock.lock(); + testBackendPid.set(resultSet.getInt(1)); + condition.signal(); + lock.unlock(); + + // Execute a long-running query. + runInvalidQuery( + testStatement, + "SELECT pg_sleep(10)", + "canceling statement due to user request" + ); + + } catch (Exception e) { + throw new RuntimeException(e); + } + }).start(); + + // Wait for backend process id of the test connection. + lock.lock(); + while (testBackendPid.get() < 0) { + condition.await(); + } + lock.unlock(); + + // Allow enough time for the long-running query to start, but not finish. + Thread.sleep(500); + + // Other roles cannot cancel the test user's query. + withRoles(statement, roles.excluding("pg_signal_backend"), () -> runInvalidQuery( + statement, + "SELECT pg_cancel_backend(" + testBackendPid + ")", + "must be a member of the role whose query is being canceled or member" + + " of pg_signal_backend" + )); + + // pg_signal_backend can cancel the query. + withRole(statement, "pg_signal_backend", + () -> statement.execute("SELECT pg_cancel_backend(" + testBackendPid + ")")); + + // No default roles can use these signals. + withRoles(statement, roles, () -> { + runInvalidQuery(statement, "SELECT pg_reload_conf()", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT pg_rotate_logfile()", PERMISSION_DENIED); + }); + + /* + * pg_read_server_files + */ + + // Grant execute on pg_read_file to all. + statement.execute( + "GRANT EXECUTE ON FUNCTION pg_read_file(text, bigint, bigint, boolean) TO PUBLIC"); + + // Create table to copy into. + statement.execute("CREATE TABLE copy_test(id int)"); + statement.execute("GRANT ALL ON TABLE copy_test TO PUBLIC"); + + withRole(statement, "pg_read_server_files", () -> { + // Can read files by absolute path. + statement.execute("SELECT pg_read_file('/non/existent/file', 0, 0, true)"); + + // Can copy from files. + statement.execute("COPY copy_test FROM '/dev/null'"); + }); + + withRoles(statement, roles.excluding("pg_read_server_files"), () -> { + // Other roles cannot read files by absolute path. + runInvalidQuery( + statement, + "SELECT pg_read_file('/non/existent/file', 0, 0, true)", + "absolute path not allowed" + ); + + // Other roles cannot copy from files. + runInvalidQuery( + statement, + "COPY copy_test FROM '/dev/null'", + "must be superuser or a member of the pg_read_server_files role to COPY from a file" + ); + }); + + /* + * pg_write_server_files + */ + + withRole(statement, "pg_write_server_files", () -> { + // Can copy to files. + statement.execute("COPY (SELECT * FROM pg_roles) TO '/dev/null'"); + }); + + withRoles(statement, roles.excluding("pg_write_server_files"), () -> { + // Other roles cannot copy to files. + runInvalidQuery( + statement, + "COPY (SELECT * FROM pg_roles) TO '/dev/null'", + "must be superuser or a member of the pg_write_server_files role to COPY to a file" + ); + }); + + /* + * pg_execute_server_program + */ + + // Create table to copy into. + statement.execute("CREATE TABLE files(filename text)"); + statement.execute("GRANT ALL ON TABLE files TO PUBLIC"); + + withRole(statement, "pg_execute_server_program", () -> { + // Can execute a command on the server. + statement.execute("COPY files FROM PROGRAM 'ls /usr/bin'"); + }); + + withRoles(statement, roles.excluding("pg_execute_server_program"), () -> { + // Other users cannot execute commands. + runInvalidQuery( + statement, + "COPY files FROM PROGRAM 'ls /usr/bin'", + "must be superuser or a member of the pg_execute_server_program role to COPY to " + + "or from an external program" + ); + }); + + /* + * pg_monitor + */ + + withRole(statement, "pg_monitor", () -> { + // Can execute monitoring functions. + statement.execute("SELECT pg_ls_waldir()"); + + // Member of pg_read_all_settings, pg_read_all_stats, and pg_stat_scan_tables. + statement.execute("SET ROLE pg_read_all_settings"); + statement.execute("SET ROLE pg_read_all_stats"); + statement.execute("SET ROLE pg_stat_scan_tables"); + + // Not a member of any other roles. + runInvalidQuery(statement, "SET ROLE pg_signal_backend", PERMISSION_DENIED); + runInvalidQuery(statement, "SET ROLE pg_read_server_files", PERMISSION_DENIED); + runInvalidQuery(statement, "SET ROLE pg_write_server_files", PERMISSION_DENIED); + runInvalidQuery(statement, "SET ROLE pg_execute_server_program", PERMISSION_DENIED); + }); + + // Other roles cannot use monitoring functions. + withRoles(statement, roles.excluding("pg_monitor"), + () -> runInvalidQuery(statement, "SELECT pg_ls_waldir()", PERMISSION_DENIED)); + } + } + + @Test + public void testAlterRoleConfiguration() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role LOGIN"); + + try (Connection connection1 = newConnectionBuilder().setUser("test_role").connect(); + Statement statement1 = connection1.createStatement()) { + assertQuery(statement1, "SHOW search_path", new Row("\"$user\", public")); + + // Set configuration variable for "test_role". + statement.execute("ALTER ROLE test_role SET search_path TO 'some path'"); + + waitForTServerHeartbeat(); + + // Change is not visible for currently open sessions. + assertQuery(statement1, "SHOW search_path", new Row("\"$user\", public")); + } + + try (Connection connection1 = newConnectionBuilder().setUser("test_role").connect(); + Statement statement1 = connection1.createStatement()) { + // Change is visible in new sessions. + assertQuery(statement1, "SHOW search_path", new Row("\"some path\"")); + } + + try (Connection connection1 = newConnectionBuilder().connect(); + Statement statement1 = connection1.createStatement()) { + withRole(statement1, "test_role", () -> { + // Change is not visible if we didn't login as "test_role". + assertQuery(statement1, "SHOW search_path", new Row("\"$user\", public")); + }); + } + + statement.execute("ALTER ROLE test_role RESET search_path"); + + try (Connection connection1 = newConnectionBuilder().setUser("test_role").connect(); + Statement statement1 = connection1.createStatement()) { + // Search path is reset. + assertQuery(statement1, "SHOW search_path", new Row("\"$user\", public")); + } + } + } + + @Test + public void testAlterRoleConfigurationInDatabase() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role LOGIN"); + + // Set configuration variable for "test_role" in "postgres". + statement.execute("ALTER ROLE test_role IN DATABASE postgres SET search_path TO 'some path'"); + + try (Connection connection1 = newConnectionBuilder().setUser("test_role").connect(); + Statement statement1 = connection1.createStatement()) { + // Change is visible in postgres database. + assertQuery(statement1, "SHOW search_path", new Row("\"some path\"")); + } + + statement.execute("CREATE DATABASE tdb"); + statement.execute("ALTER DATABASE tdb OWNER TO test_role"); + + try (Connection connection1 = newConnectionBuilder().setDatabase("tdb") + .setUser("test_role").connect(); + Statement statement1 = connection1.createStatement()) { + // Change is not visible in other databases. + assertQuery(statement1, "SHOW search_path", new Row("\"$user\", public")); + } + } + } + + @Test + public void testAlterRoleConfigurationForAll() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role LOGIN"); + statement.execute("CREATE ROLE other_role LOGIN"); + + // Set configuration variable for all roles. + statement.execute("ALTER ROLE ALL SET search_path TO 'some path'"); + + // Change is visible for both users. + try (Connection connection1 = newConnectionBuilder().setUser("test_role").connect(); + Statement statement1 = connection1.createStatement()) { + assertQuery(statement1, "SHOW search_path", new Row("\"some path\"")); + } + try (Connection connection1 = newConnectionBuilder().setUser("other_role").connect(); + Statement statement1 = connection1.createStatement()) { + assertQuery(statement1, "SHOW search_path", new Row("\"some path\"")); + } + } + } + + @Test + public void testGroupMembershipCycles() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE r1"); + statement.execute("CREATE ROLE r2 ROLE r1"); + statement.execute("CREATE ROLE r3 ROLE r2"); + statement.execute("CREATE ROLE r4 ROLE r3"); + + runInvalidQuery(statement, "GRANT r1 TO r4", "role \"r1\" is a member of role \"r4\""); + } + } + + @Test + public void testDatabasePrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE test_db"); + ConnectionBuilder tdbConnector = newConnectionBuilder().setDatabase("test_db"); + + // Remove any default privileges. + statement.execute("REVOKE ALL ON DATABASE test_db FROM PUBLIC"); + + /* + * CREATE + */ + + statement.execute("CREATE ROLE create_role LOGIN"); + statement.execute("GRANT CREATE ON DATABASE test_db TO create_role"); + + try (Connection connection1 = tdbConnector.newBuilder().connect(); + Statement statement1 = connection1.createStatement()) { + withRole(statement1, "create_role", () -> { + // Can create schemas. + statement1.execute("CREATE SCHEMA test_schema"); + + // Cannot create temporary tables. + runInvalidQuery(statement1, "CREATE TEMP TABLE test_table(id int)", PERMISSION_DENIED); + }); + } + + // Cannot connect directly to the database. + try (Connection ignored = tdbConnector.newBuilder().setUser("create_role").connect()) { + fail("Expected connection attempt to fail"); + } catch (SQLException sqle) { + assertThat(sqle.getMessage(), CoreMatchers.containsString(PERMISSION_DENIED)); + } + + /* + * CONNECT + */ + + statement.execute("CREATE ROLE connect_role LOGIN"); + statement.execute("GRANT CONNECT ON DATABASE test_db TO connect_role"); + + // Can connect directly to the database. + try (Connection connection1 = tdbConnector.newBuilder().setUser("connect_role").connect(); + Statement statement1 = connection1.createStatement()) { + // Cannot create schemas. + runInvalidQuery(statement1, "CREATE SCHEMA other_schema", PERMISSION_DENIED); + + // Cannot create temporary tables. + runInvalidQuery(statement1, "CREATE TEMP TABLE test_table(id int)", PERMISSION_DENIED); + } + + /* + * TEMP + */ + + statement.execute("CREATE ROLE temp_role LOGIN"); + statement.execute("GRANT TEMP ON DATABASE test_db TO temp_role"); + + try (Connection connection1 = tdbConnector.newBuilder().connect(); + Statement statement1 = connection1.createStatement()) { + withRole(statement1, "temp_role", () -> { + // Can create temporary tables. + statement.execute("CREATE TEMP TABLE test_table(id int)"); + + // Cannot create schemas. + runInvalidQuery(statement1, "CREATE SCHEMA other_schema", PERMISSION_DENIED); + }); + } + + // Cannot connect directly to the database. + try (Connection ignored = tdbConnector.newBuilder().setUser("temp_role").connect()) { + fail("Expected connection attempt to fail"); + } catch (SQLException sqle) { + assertThat(sqle.getMessage(), CoreMatchers.containsString(PERMISSION_DENIED)); + } + } + } + + @Test + public void testSchemaPrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE SCHEMA test_schema"); + + // Remove any default privileges. + statement.execute("REVOKE ALL ON SCHEMA test_schema FROM PUBLIC"); + + /* + * CREATE + */ + + statement.execute("CREATE ROLE create_role"); + statement.execute("GRANT CREATE ON SCHEMA test_schema TO create_role"); + + withRole(statement, "create_role", () -> { + // Can create objects in schema. + statement.execute("CREATE TABLE test_schema.test_table(id int)"); + + // Cannot rename objects without USAGE. + runInvalidQuery( + statement, + "ALTER TABLE test_schema.test_table RENAME TO new_table", + PERMISSION_DENIED + ); + + withRole(statement, TEST_PG_USER, () -> + statement.execute("GRANT USAGE ON SCHEMA test_schema TO create_role")); + + // Can rename owned objects. + statement.execute("ALTER TABLE test_schema.test_table RENAME TO new_table"); + + withRole(statement, TEST_PG_USER, () -> + statement.execute("REVOKE CREATE ON SCHEMA test_schema FROM create_role")); + + // Revoking CREATE removes rename privileges, even if table is owned. + runInvalidQuery( + statement, + "ALTER TABLE test_schema.new_table RENAME TO test_table", + PERMISSION_DENIED + ); + }); + + /* + * USAGE + */ + + statement.execute("CREATE ROLE usage_role"); + statement.execute("GRANT USAGE ON SCHEMA test_schema TO usage_role"); + + withRole(statement, "usage_role", () -> { + // Cannot create objects in schema. + runInvalidQuery( + statement, + "CREATE TABLE test_schema.other_table(id int)", + PERMISSION_DENIED + ); + + withRole(statement, "create_role", () -> + statement.execute("GRANT SELECT ON TABLE test_schema.new_table TO usage_role")); + + // Can use existing objects (with permission). + statement.execute("SELECT * FROM test_schema.new_table"); + }); + } + } + + @Test + public void testDomainPrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE DOMAIN nn_int AS int NOT NULL"); + + // Revoke any default privileges. + statement.execute("REVOKE ALL ON DOMAIN nn_int FROM PUBLIC"); + + statement.execute("CREATE ROLE test_role"); + + withRole(statement, "test_role", () -> { + // Cannot create objects using domain without USAGE. + runInvalidQuery(statement, "CREATE TABLE test_table(id nn_int)", PERMISSION_DENIED); + }); + + statement.execute("GRANT USAGE ON DOMAIN nn_int TO test_role"); + + withRole(statement, "test_role", () -> { + // Can create objects using domain after grant. + statement.execute("CREATE TABLE test_table(id nn_int)"); + }); + + statement.execute("REVOKE USAGE ON DOMAIN nn_int FROM test_role"); + + withRole(statement, "test_role", () -> { + // Can still select type without USAGE. + statement.execute("SELECT * FROM test_table"); + }); + } + } + + @Test + public void testSequencePrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE SEQUENCE test_seq"); + + // Revoke any default privileges. + statement.execute("REVOKE ALL ON SEQUENCE test_seq FROM PUBLIC"); + + /* + * USAGE + */ + + statement.execute("CREATE ROLE usage_role"); + statement.execute("GRANT USAGE ON SEQUENCE test_seq TO usage_role"); + + withRole(statement, "usage_role", () -> { + // Can use "currval" and "nextval". + statement.execute("SELECT nextval('test_seq')"); + statement.execute("SELECT currval('test_seq')"); + + // Cannot use "setval". + runInvalidQuery(statement, "SELECT setval('test_seq', 3)", PERMISSION_DENIED); + + // Cannot select. + runInvalidQuery(statement, "SELECT * FROM test_seq", PERMISSION_DENIED); + }); + + /* + * SELECT + */ + + statement.execute("CREATE ROLE select_role"); + statement.execute("GRANT SELECT ON SEQUENCE test_seq TO select_role"); + + withRole(statement, "select_role", () -> { + // Can use "currval". + statement.execute("SELECT currval('test_seq')"); + + // Cannot use "nextval" or "setval". + runInvalidQuery(statement, "SELECT nextval('test_seq')", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT setval('test_seq', 3)", PERMISSION_DENIED); + + // Can select sequence. + statement.execute("SELECT * FROM test_seq"); + }); + + /* + * UPDATE + */ + + statement.execute("CREATE ROLE update_role"); + statement.execute("GRANT UPDATE ON SEQUENCE test_seq TO update_role"); + + withRole(statement, "update_role", () -> { + // Can use "nextval" and "setval". + statement.execute("SELECT nextval('test_seq')"); + statement.execute("SELECT setval('test_seq', 3)"); + + // Cannot use "curval". + runInvalidQuery(statement, "SELECT currval('test_seq')", PERMISSION_DENIED); + + // Cannot select. + runInvalidQuery(statement, "SELECT * FROM test_seq", PERMISSION_DENIED); + }); + } + } + + @Test + public void testViewPrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + statement.execute("CREATE TABLE test_view_back(a int, b int, c int, PRIMARY KEY (a))"); + statement.execute("CREATE VIEW test_view AS SELECT a, c FROM test_view_back"); + + /* + * SELECT + */ + + statement.execute("GRANT SELECT ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Select is allowed. + statement.execute("SELECT * FROM test_view"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_view(a, c) VALUES (1, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * INSERT + */ + + statement.execute("GRANT INSERT ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Insert is allowed. + statement.execute("INSERT INTO test_view(a, c) VALUES (1, 3)"); + + // Others are not. + runInvalidQuery(statement, "SELECT * FROM test_view", PERMISSION_DENIED); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * UPDATE + */ + + statement.execute("GRANT UPDATE ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Update is allowed. + statement.execute("UPDATE test_view SET c = 2"); + + // Others are not allowed. + runInvalidQuery( + statement, + "INSERT INTO test_view(a, c) VALUES (1, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * DELETE, SELECT + */ + + statement.execute("GRANT DELETE, SELECT ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Delete with selection is allowed. + statement.execute("DELETE FROM test_view WHERE a = 2"); + + // Select is allowed. + statement.execute("SELECT * FROM test_view"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_view(a, c) VALUES (1, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * UPDATE, SELECT + */ + + statement.execute("GRANT UPDATE ON TABLE test_view TO test_role"); + statement.execute("GRANT SELECT ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Update with selection is allowed. + statement.execute("UPDATE test_view SET c = 2 WHERE a = 1"); + + // Others are not allowed. + runInvalidQuery( + statement, + "INSERT INTO test_view(a, c) VALUES (1, 3)", + PERMISSION_DENIED + ); + }); + + statement.execute("REVOKE SELECT ON TABLE test_view FROM test_role"); + withRole(statement, "test_role", () -> { + // Can no longer perform updates with selection. + runInvalidQuery( + statement, + "UPDATE test_view SET c = 2 WHERE a = 1", + PERMISSION_DENIED + ); + + // Can still perform simple updates. + statement.execute("UPDATE test_view SET c = 2"); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * ALL + */ + + statement.execute("GRANT ALL ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // All are allowed. + statement.execute("SELECT * FROM test_view"); + statement.execute("INSERT INTO test_view(a, c) VALUES (2, 3)"); + statement.execute("UPDATE test_view SET c = 2"); + statement.execute("DELETE FROM test_view WHERE a = 2"); + }); + } + } + + @Test + public void testViewColumnPrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + statement.execute("CREATE TABLE test_view_back(a int, b int, c int, PRIMARY KEY (a))"); + statement.execute("CREATE VIEW test_view AS SELECT a, b, c FROM test_view_back"); + + /* + * SELECT + */ + + statement.execute("GRANT SELECT(a) ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Select from "a" is allowed. + statement.execute("SELECT a FROM test_view"); + + // Others are not. + runInvalidQuery(statement, "SELECT b FROM test_view", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT c FROM test_view", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM test_view", PERMISSION_DENIED); + runInvalidQuery( + statement, + "INSERT INTO test_view(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + statement.execute("GRANT SELECT(a, b, c) ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Select is allowed. + statement.execute("SELECT * FROM test_view"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_view(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + + // Revoke a single column. + statement.execute("REVOKE SELECT(b) ON TABLE test_view FROM test_role"); + withRole(statement, "test_role", () -> { + // Select from "a" and "c" are allowed. + statement.execute("SELECT a FROM test_view"); + statement.execute("SELECT c FROM test_view"); + statement.execute("SELECT a, c FROM test_view"); + + // Others are not. + runInvalidQuery(statement, "SELECT b FROM test_view", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM test_view", PERMISSION_DENIED); + runInvalidQuery( + statement, + "INSERT INTO test_view(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * INSERT + */ + + statement.execute("GRANT INSERT(a) ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Insert "a" is allowed. + statement.execute("INSERT INTO test_view(a) VALUES (2)"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_view(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "INSERT INTO test_view(b) VALUES (4)", + PERMISSION_DENIED + ); + runInvalidQuery(statement, "SELECT * FROM test_view", PERMISSION_DENIED); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * UPDATE + */ + + statement.execute("GRANT UPDATE(c) ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Update is allowed. + statement.execute("UPDATE test_view SET c = 2"); + + // Others are not allowed. + runInvalidQuery( + statement, + "UPDATE test_view SET c = 2 WHERE a = 1", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "UPDATE test_view SET a = 2, b = 2, c = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "UPDATE test_view SET b = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "INSERT INTO test_view(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * UPDATE, SELECT + */ + + statement.execute("GRANT UPDATE(c), SELECT(a) ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // Update with selection is allowed. + statement.execute("UPDATE test_view SET c = 2 WHERE a = 1"); + + // Others are not allowed. + runInvalidQuery( + statement, + "UPDATE test_view SET a = 2, b = 2, c = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "UPDATE test_view SET b = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "INSERT INTO test_view(c) VALUES (2)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_view FROM test_role"); + + /* + * ALL + */ + + statement.execute("GRANT ALL(a, b) ON TABLE test_view TO test_role"); + withRole(statement, "test_role", () -> { + // All on "a" and "b" are allowed. + statement.execute("SELECT a FROM test_view"); + statement.execute("INSERT INTO test_view(a, b) VALUES (3, 4)"); + statement.execute("UPDATE test_view SET a = 2 WHERE b = 3"); + + // Others are not. + runInvalidQuery(statement, "SELECT b, c FROM test_view", PERMISSION_DENIED); + runInvalidQuery( + statement, + "INSERT INTO test_view(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + runInvalidQuery(statement, "UPDATE test_view SET c = 3", PERMISSION_DENIED); + }); + } + } + + @Test + public void testTemporaryTablePrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + statement.execute("CREATE TEMP TABLE test_table(a int, b int, c int, PRIMARY KEY (a))"); + + /* + * SELECT + */ + + statement.execute("GRANT SELECT ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Select is allowed. + statement.execute("SELECT * FROM test_table"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * INSERT + */ + + statement.execute("GRANT INSERT ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Insert is allowed. + statement.execute("INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)"); + + // Others are not. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * UPDATE + */ + + statement.execute("GRANT UPDATE ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Update is allowed. + statement.execute("UPDATE test_table SET c = 2"); + + // Others are not allowed. + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * DELETE, SELECT + */ + + statement.execute("GRANT DELETE, SELECT ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Delete with selection is allowed. + statement.execute("DELETE FROM test_table WHERE a = 2"); + + // Select is allowed. + statement.execute("SELECT * FROM test_table"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * TRUNCATE + */ + + statement.execute("GRANT TRUNCATE ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Truncate is allowed. + statement.execute("TRUNCATE TABLE test_table"); + + // Others are not. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * UPDATE, SELECT + */ + + statement.execute("GRANT UPDATE ON TABLE test_table TO test_role"); + statement.execute("GRANT SELECT ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Update with selection is allowed. + statement.execute("UPDATE test_table SET c = 2 WHERE a = 1"); + + // Others are not allowed. + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + + statement.execute("REVOKE SELECT ON TABLE test_table FROM test_role"); + withRole(statement, "test_role", () -> { + // Can no longer perform updates with selection. + runInvalidQuery( + statement, + "UPDATE test_table SET c = 2 WHERE a = 1", + PERMISSION_DENIED + ); + + // Can still perform simple updates. + statement.execute("UPDATE test_table SET c = 2"); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * ALL + */ + + statement.execute("GRANT ALL ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // All are allowed. + statement.execute("SELECT * FROM test_table"); + statement.execute("INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)"); + statement.execute("UPDATE test_table SET c = 2"); + statement.execute("DELETE FROM test_table WHERE a = 2"); + }); + } + } + + @Test + public void testTemporaryTableColumnPrivileges() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + statement.execute("CREATE TEMP TABLE test_table(a int, b int, c int, PRIMARY KEY (a))"); + + /* + * SELECT + */ + + statement.execute("GRANT SELECT(a) ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Select from "a" is allowed. + statement.execute("SELECT a FROM test_table"); + + // Others are not. + runInvalidQuery(statement, "SELECT b FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT c FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + statement.execute("GRANT SELECT(a, b, c) ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Select is allowed. + statement.execute("SELECT * FROM test_table"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + + // Revoke a single column. + statement.execute("REVOKE SELECT(b) ON TABLE test_table FROM test_role"); + withRole(statement, "test_role", () -> { + // Select from "a" and "c" are allowed. + statement.execute("SELECT a FROM test_table"); + statement.execute("SELECT c FROM test_table"); + statement.execute("SELECT a, c FROM test_table"); + + // Others are not. + runInvalidQuery(statement, "SELECT b FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * INSERT + */ + + statement.execute("GRANT INSERT(a) ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Insert "a" is allowed. + statement.execute("INSERT INTO test_table(a) VALUES (2)"); + + // Others are not. + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "INSERT INTO test_table(b) VALUES (4)", + PERMISSION_DENIED + ); + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * UPDATE + */ + + statement.execute("GRANT UPDATE(c) ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Update is allowed. + statement.execute("UPDATE test_table SET c = 2"); + + // Others are not allowed. + runInvalidQuery( + statement, + "UPDATE test_table SET c = 2 WHERE a = 1", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "UPDATE test_table SET a = 2, b = 2, c = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "UPDATE test_table SET b = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * UPDATE, SELECT + */ + + statement.execute("GRANT UPDATE(c), SELECT(a) ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // Update with selection is allowed. + statement.execute("UPDATE test_table SET c = 2 WHERE a = 1"); + + // Others are not allowed. + runInvalidQuery( + statement, + "UPDATE test_table SET a = 2, b = 2, c = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "UPDATE test_table SET b = 2", + PERMISSION_DENIED + ); + runInvalidQuery( + statement, + "INSERT INTO test_table(c) VALUES (2)", + PERMISSION_DENIED + ); + }); + statement.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + /* + * ALL + */ + + statement.execute("GRANT ALL(a, b) ON TABLE test_table TO test_role"); + withRole(statement, "test_role", () -> { + // All on "a" and "b" are allowed. + statement.execute("SELECT a FROM test_table"); + statement.execute("INSERT INTO test_table(a, b) VALUES (3, 4)"); + statement.execute("UPDATE test_table SET a = 2 WHERE b = 3"); + + // Others are not. + runInvalidQuery(statement, "SELECT b, c FROM test_table", PERMISSION_DENIED); + runInvalidQuery( + statement, + "INSERT INTO test_table(a, b, c) VALUES (1, 2, 3)", + PERMISSION_DENIED + ); + runInvalidQuery(statement, "UPDATE test_table SET c = 3", PERMISSION_DENIED); + }); + } + } + + @Test + public void testRevokeAdminOption() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE some_role"); + statement.execute("CREATE ROLE some_group ADMIN some_role"); + + statement.execute("CREATE ROLE other_role"); + + withRole(statement, "some_role", () -> { + // "some_role" has admin option in "some_group". + statement.execute("GRANT some_group TO other_role"); + }); + + statement.execute("REVOKE ADMIN OPTION FOR some_group FROM some_role"); + + withRole(statement, "some_role", () -> { + // "some_role" no longer has admin option in "some_group". + runInvalidQuery( + statement, + "GRANT some_group TO other_role", + "must have admin option on role \"some_group\"" + ); + + // "some_role" is still a member of "some_group". + statement.execute("SET ROLE some_group"); + }); + + withRole(statement, "other_role", () -> { + // "other_role" is still a member of "some_group". + statement.execute("SET ROLE some_group"); + }); + } + } + + @Test + public void testCascadingRevoke() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE a"); + statement.execute("CREATE ROLE b"); + + // Create table and grant privileges to role "a". + statement.execute("CREATE TABLE test_table(id int)"); + statement.execute("GRANT ALL ON TABLE test_table TO a WITH GRANT OPTION"); + + withRole(statement, "a", () -> { + // Grant table to "b" from "a". + statement.execute("GRANT ALL ON TABLE test_table TO b"); + }); + + withRole(statement, "b", () -> { + // Role "b" has indirectly received privileges for "test_table". + statement.execute("SELECT * FROM test_table"); + }); + + // Revoking privileges from "a" without cascading results in an error. + runInvalidQuery( + statement, + "REVOKE ALL ON TABLE test_table FROM a", + "dependent privileges exist" + ); + + // Revoke privileges from "a" and cascade to any privileges granted by "a". + statement.execute("REVOKE ALL ON TABLE test_table FROM a CASCADE"); + + withRole(statement, "a", () -> { + // Role "a" no longer has privileges. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + }); + + withRole(statement, "b", () -> { + // Role "b" has also lost its privileges. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + }); + } + } + + @Test + public void testReferencesOnTables() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE has_references"); + statement.execute("CREATE ROLE no_references"); + + statement.execute("CREATE TABLE customers(id int PRIMARY KEY)"); + statement.execute("GRANT REFERENCES ON TABLE customers TO has_references"); + + withRole(statement, "has_references", () -> { + // "has_references" can create a table referencing a column in "customers". + statement.execute( + "CREATE TABLE orders(id int PRIMARY KEY, cid int REFERENCES customers(id))"); + }); + + withRole(statement, "no_references", () -> { + // "no_references" user cannot create a table referencing a column in "customers". + runInvalidQuery( + statement, + "CREATE TABLE emails(id int PRIMARY KEY, cid int REFERENCES customers(id))", + PERMISSION_DENIED + ); + }); + } + } + + @Test + public void testReferencesOnColumns() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE has_references"); + statement.execute("CREATE ROLE no_references"); + + statement.execute("CREATE TABLE table1(id1 int PRIMARY KEY, id2 int UNIQUE)"); + statement.execute("GRANT REFERENCES(id2) ON TABLE table1 TO has_references"); + + statement.execute("CREATE TABLE table2(id1 int PRIMARY KEY, id2 int UNIQUE, id3 int UNIQUE)"); + statement.execute("GRANT REFERENCES(id1, id3) ON TABLE table2 TO has_references"); + + withRole(statement, "has_references", () -> { + // Can reference single granted column from table1. + statement.execute( + "CREATE TABLE test_table(id int PRIMARY KEY, cid int REFERENCES table1(id2))"); + statement.execute("DROP TABLE test_table"); + + // Can reference single granted column from table2. + statement.execute( + "CREATE TABLE test_table(id int PRIMARY KEY, cid int REFERENCES table2(id1))"); + statement.execute("DROP TABLE test_table"); + + // Can reference multiple granted columns from table2. + statement.execute("CREATE TABLE test_table(id int PRIMARY KEY," + + " cid int REFERENCES table2(id1)," + + " did int REFERENCES table2(id3))"); + statement.execute("DROP TABLE test_table"); + + // Cannot reference columns which were not granted. + runInvalidQuery( + statement, + "CREATE TABLE test_table(id int PRIMARY KEY, cid int REFERENCES table1(id1))", + PERMISSION_DENIED + ); + // TODO remove this DROP once #1383 is implemented. + statement.execute("DROP TABLE test_table"); + runInvalidQuery( + statement, + "CREATE TABLE test_table(id int PRIMARY KEY," + + " cid int REFERENCES table1(id1)," + + " did int REFERENCES table1(id2))", + PERMISSION_DENIED + ); + // TODO remove this DROP once #1383 is implemented. + statement.execute("DROP TABLE test_table"); + runInvalidQuery( + statement, + "CREATE TABLE test_table(id int PRIMARY KEY," + + " cid int REFERENCES table2(id3)," + + " did int REFERENCES table2(id2))", + PERMISSION_DENIED + ); + // TODO remove this DROP once #1383 is implemented. + statement.execute("DROP TABLE test_table"); + }); + + withRole(statement, "no_references", () -> { + // "no_references" role cannot create tables referencing columns in either table. + runInvalidQuery( + statement, + "CREATE TABLE test_table(id int PRIMARY KEY, cid int REFERENCES table1(id2))", + PERMISSION_DENIED + ); + // TODO remove this DROP once #1383 is implemented. + statement.execute("DROP TABLE test_table"); + runInvalidQuery( + statement, + "CREATE TABLE test_table(id int PRIMARY KEY, cid int REFERENCES table2(id3))", + PERMISSION_DENIED + ); + }); + } + } + + @Test + public void testForeignKeyConstraintACLCheck() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + statement.execute("CREATE TABLE test_table(id1 int PRIMARY KEY, id2 int UNIQUE)"); + statement.execute("GRANT REFERENCES(id1), SELECT ON TABLE test_table TO test_role"); + + withRole(statement, "test_role", () -> { + statement.execute( + "CREATE TABLE reference_table(id int PRIMARY KEY, cid int)"); + + runInvalidQuery( + statement, + "ALTER TABLE reference_table ADD FOREIGN KEY (cid) REFERENCES test_table(id2)", + PERMISSION_DENIED + ); + + statement.execute( + "ALTER TABLE reference_table ADD FOREIGN KEY (cid) REFERENCES test_table(id1)"); + }); + } + } + + @Test + public void testOnAllTablesInSchema() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + // Create table outside schema. + statement.execute("CREATE TABLE test1(id int)"); + + statement.execute("CREATE SCHEMA ts"); + statement.execute("GRANT USAGE ON SCHEMA ts TO test_role"); + + // Create tables inside schema. + statement.execute("CREATE TABLE ts.test1(id int)"); + statement.execute("CREATE TABLE ts.test2(id int)"); + + statement.execute("GRANT SELECT ON ALL TABLES IN SCHEMA ts TO test_role"); + + // Create table inside schema, after grant has occurred. + statement.execute("CREATE TABLE ts.test3(id int)"); + + withRole(statement, "test_role", () -> { + // Only tables which were in the schema before the grant have privileges. + runInvalidQuery(statement, "SELECT * FROM test1", PERMISSION_DENIED); + statement.execute("SELECT * FROM ts.test1"); + statement.execute("SELECT * FROM ts.test2"); + runInvalidQuery(statement, "SELECT * FROM ts.test3", PERMISSION_DENIED); + + // Tables only have select privileges. + runInvalidQuery(statement, "TRUNCATE ts.test1", PERMISSION_DENIED); + }); + } + } + + @Test + public void testPrivilegesAsSumOfGrantAcrossMembership() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + statement.execute("CREATE ROLE test_group ROLE test_role"); + + // Create function. + statement.execute("CREATE FUNCTION test() RETURNS int AS 'SELECT 1' LANGUAGE SQL"); + + // Grant EXECUTE to "test_role", and both groups it is a member of. + statement.execute("GRANT ALL ON FUNCTION test() TO test_role"); + statement.execute("GRANT ALL ON FUNCTION test() TO test_group"); + statement.execute("GRANT ALL ON FUNCTION test() TO PUBLIC"); + + // Revoking from "test_role" does not change sum of grant privileges. + statement.execute("REVOKE ALL ON FUNCTION test() FROM test_role"); + withRole(statement, "test_role", () -> { + // Function can still be executed. + statement.execute("SELECT test()"); + }); + + // Revoking from PUBLIC does not change sum of grant privileges. + statement.execute("REVOKE ALL ON FUNCTION test() FROM PUBLIC"); + withRole(statement, "test_role", () -> { + // Function can still be executed. + statement.execute("SELECT test()"); + }); + + // Revoking from "test_group" removes all granted privileges. + statement.execute("REVOKE ALL ON FUNCTION test() FROM test_group"); + withRole(statement, "test_role", () -> { + // Function can no longer be executed. + runInvalidQuery(statement, "SELECT test()", PERMISSION_DENIED); + }); + } + } + + @Test + public void testOnAllFunctionsInSchema() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role"); + + // By default, don't allow public to execute new functions. + statement.execute( + "ALTER DEFAULT PRIVILEGES REVOKE EXECUTE ON FUNCTIONS FROM PUBLIC"); + + // Create function outside schema. + statement.execute("CREATE FUNCTION test1() RETURNS int AS 'SELECT 1' LANGUAGE SQL"); + + statement.execute("CREATE SCHEMA ts"); + statement.execute("GRANT USAGE ON SCHEMA ts TO test_role"); + + // Create functions inside schema. + statement.execute("CREATE FUNCTION ts.test1() RETURNS int AS 'SELECT 1' LANGUAGE SQL"); + statement.execute("CREATE FUNCTION ts.test2() RETURNS int AS 'SELECT 1' LANGUAGE SQL"); + + statement.execute("GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA ts TO test_role"); + + // Create function inside schema, after grant has occurred. + statement.execute("CREATE FUNCTION ts.test3() RETURNS int AS 'SELECT 1' LANGUAGE SQL"); + + withRole(statement, "test_role", () -> { + // Only functions which were in the schema before the grant have privileges. + runInvalidQuery(statement, "SELECT test1()", PERMISSION_DENIED); + statement.execute("SELECT ts.test1()"); + statement.execute("SELECT ts.test2()"); + runInvalidQuery(statement, "SELECT ts.test3()", PERMISSION_DENIED); + }); + } + } + + @Test + public void testAlterObjectOwner() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE old_user"); + statement.execute("CREATE ROLE new_user"); + statement.execute("CREATE ROLE some_group ROLE old_user"); + + // Create a schema and grant access. + statement.execute("CREATE SCHEMA ts"); + statement.execute("GRANT ALL ON SCHEMA ts TO old_user, new_user"); + + withRole(statement, "old_user", () -> { + // Create some tables. + statement.execute("CREATE TABLE test_table(id int)"); + statement.execute("CREATE TABLE ts.test_table(id int)"); + + // Transfer ownership of table to parent. + statement.execute("ALTER TABLE test_table OWNER TO some_group"); + + // Can still select, since "old_user" is a member of "some_group" (the new owner). + statement.execute("SELECT * FROM test_table"); + + // Can still perform owner-only operations, since "old_user" is a member of "some_group". + statement.execute("DROP TABLE test_table"); + + // Cannot assign ownership to a role we are not a member of. + runInvalidQuery( + statement, + "ALTER TABLE ts.test_table OWNER TO new_user", + "must be member of role \"new_user\"" + ); + }); + + // Superusers can assign ownership to any role. + statement.execute("ALTER TABLE ts.test_table OWNER TO new_user"); + + // This should be a no-op. + statement.execute("DROP OWNED BY old_user CASCADE"); + + withRole(statement, "old_user", () -> { + // Old user has lost all privileges on the table. + runInvalidQuery(statement, "SELECT * FROM ts.test_table", PERMISSION_DENIED); + }); + + withRole(statement, "new_user", () -> { + // New role has ownership privileges on the table. + statement.execute("DROP TABLE ts.test_table"); + }); + } + } + + @Test + public void testAlterDatabaseOwner() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE su1 CREATEDB"); + statement.execute("CREATE ROLE su2 LOGIN CREATEDB"); + + statement.execute("GRANT su1 TO su2"); + + withRole(statement, "su1", () -> { + // Create a database owned by "su1". + statement.execute("CREATE DATABASE su1_db"); + }); + + // Alter database owner externally. + statement.execute("ALTER DATABASE su1_db OWNER TO su2"); + + withRole(statement, "su2", () -> { + // Create a database owned by "su2". + statement.execute("CREATE DATABASE su2_db"); + }); + } + + try (Connection connection = newConnectionBuilder().setDatabase("su2_db") + .setUser("su2").connect(); + Statement statement = connection.createStatement()) { + // Alter database owner from within database. + statement.execute("ALTER DATABASE su2_db OWNER TO su1"); + } + + try (Statement statement = connection.createStatement()) { + // Allow time for cache to refresh after ALTER above. + waitForTServerHeartbeat(); + + statement.execute("REVOKE su1 FROM su2"); + + // Database owners are correctly set. + withRole(statement, "su1", () -> + runInvalidQuery(statement, "DROP DATABASE su1_db", "must be owner of database su1_db")); + withRole(statement, "su2", () -> + runInvalidQuery(statement, "DROP DATABASE su2_db", "must be owner of database su2_db")); + withRole(statement, "su1", () -> + statement.execute("DROP DATABASE su2_db")); + withRole(statement, "su2", () -> + statement.execute("DROP DATABASE su1_db")); + } + } + + @Test + public void testReassignOwnedObjects() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE old_user"); + statement.execute("CREATE ROLE new_user"); + statement.execute("CREATE ROLE some_group ROLE old_user"); + + // Create a schema and grant access. + statement.execute("CREATE SCHEMA ts"); + statement.execute("GRANT ALL ON SCHEMA ts TO old_user, new_user"); + + withRole(statement, "old_user", () -> { + // Create some objects. + statement.execute("CREATE TABLE test_table(id int PRIMARY KEY)"); + statement.execute("CREATE FUNCTION ts.test() RETURNS int AS 'SELECT 1' LANGUAGE SQL"); + statement.execute("CREATE TEMP TABLE temp_table(id int)"); + statement.execute("CREATE VIEW ts.test_view AS SELECT * FROM public.test_table"); + statement.execute( + "CREATE TABLE referencing_table(id int, tid int REFERENCES test_table(id))"); + statement.execute( + "ALTER DEFAULT PRIVILEGES IN SCHEMA ts GRANT EXECUTE ON FUNCTIONS TO PUBLIC"); + + // Cannot reassign ownership to roles we are not a member of. + runInvalidQuery( + statement, + "REASSIGN OWNED BY CURRENT_USER TO new_user", + PERMISSION_DENIED + ); + + // Reassign fails because "some_group" does not have schema permissions. + runInvalidQuery( + statement, + "REASSIGN OWNED BY CURRENT_USER TO some_group", + "permission denied for schema ts" + ); + + withRole(statement, TEST_PG_USER, () -> + statement.execute("GRANT ALL ON SCHEMA ts TO some_group")); + + // Can reassign ownership after granting. + statement.execute("REASSIGN OWNED BY CURRENT_USER TO some_group"); + + // Old role still has ownership privileges, since it is a member of "some_group". + statement.execute("SELECT * FROM test_table"); + statement.execute("SELECT * FROM ts.test_view"); + statement.execute("ALTER TABLE test_table ADD COLUMN c1 int"); + + // Create new object as old_user. + statement.execute("CREATE TABLE new_table(id int)"); + }); + + withRole(statement, "some_group", () -> { + // Group has ownership privileges on original objects. + statement.execute("SELECT * FROM test_table"); + statement.execute("SELECT * FROM ts.test_view"); + statement.execute("ALTER TABLE test_table ADD COLUMN c2 int"); + + // Group does not have privileges on new table. + runInvalidQuery(statement, "SELECT * FROM new_table", PERMISSION_DENIED); + }); + + withRole(statement, "new_user", () -> { + // New user still does not have permissions on objects. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM new_table", PERMISSION_DENIED); + }); + + // Reassign ownership of both roles to the new user. + statement.execute("REASSIGN OWNED BY some_group, old_user TO new_user"); + + // This should be a no-op. + statement.execute("DROP OWNED BY some_group, old_user CASCADE"); + + withRole(statement, "new_user", () -> { + // New user has permissions on original objects and the new table. + statement.execute("SELECT * FROM test_table"); + statement.execute("SELECT * FROM ts.test_view"); + statement.execute("ALTER TABLE test_table ADD COLUMN c3 int"); + statement.execute("SELECT * FROM new_table"); + }); + + withRole(statement, "old_user", () -> { + // Old user no longer has permissions. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM new_table", PERMISSION_DENIED); + }); + + withRole(statement, "some_group", () -> { + // Group no longer has permissions. + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement, "SELECT * FROM new_table", PERMISSION_DENIED); + }); + + // REASSIGN does not remove permissions for unowned objects. + statement.execute("GRANT SELECT ON new_table TO old_user"); + statement.execute("REASSIGN OWNED BY old_user TO new_user"); + runInvalidQuery( + statement, + "DROP ROLE old_user", + "role \"old_user\" cannot be dropped because some objects depend on it" + ); + } + } + + @Test + public void testDropOwnershipOnDatabaseObjects() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE test_role SUPERUSER"); + + /* + * SIMPLE + */ + + withRole(statement, "test_role", () -> { + statement.execute("CREATE TABLE test_table(id int)"); + + // This should drop the created table. + statement.execute("DROP OWNED BY test_role"); + + statement.execute("CREATE TABLE test_table(id int)"); + }); + + // DROP OWNED does remove permissions for unowned objects. + statement.execute("CREATE TABLE new_table(id int)"); + statement.execute("GRANT SELECT ON TABLE new_table TO test_role"); + statement.execute("DROP OWNED BY test_role"); + statement.execute("DROP ROLE test_role"); + + statement.execute("CREATE ROLE test_role SUPERUSER"); + + /* + * CASCADE/RESTRICT + */ + + withRole(statement, "test_role", () -> { + statement.execute("CREATE ROLE schema_user"); + + statement.execute("CREATE SCHEMA test_schema"); + statement.execute("GRANT ALL ON SCHEMA test_schema TO schema_user"); + + withRole(statement, "schema_user", () -> { + // Create a table in "test_schema" which is not owned by "test_role". + statement.execute("CREATE TABLE test_schema.test_table(id int)"); + }); + + // DROP and DROP RESTRICT fail due to unowned table. + runInvalidQuery( + statement, + "DROP OWNED BY test_role RESTRICT", + "table test_schema.test_table depends on schema test_schema" + ); + runInvalidQuery( + statement, + "DROP OWNED BY test_role", + "table test_schema.test_table depends on schema test_schema" + ); + + // DROP CASCADE succeeds. + statement.execute("DROP OWNED BY test_role CASCADE"); + + // Schema and table were dropped. + statement.execute("CREATE SCHEMA test_schema"); + statement.execute("CREATE TABLE test_schema.test_table(id int)"); + }); + } + } + + @Test + public void testMultiDatabaseOwnershipDrop() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE USER su LOGIN SUPERUSER"); + withRole(statement, "su", () -> { + // Create a database owned by "su". + statement.execute("CREATE DATABASE su_db"); + + // Create object in default database. + statement.execute("CREATE TABLE test_table(id int)"); + }); + } + + try (Connection connection = newConnectionBuilder().setDatabase("su_db") + .setUser("su").connect(); + Statement statement = connection.createStatement()) { + // Create object in "su_db". + statement.execute("CREATE TABLE test_table(id int)"); + + statement.execute("DROP OWNED BY su CASCADE"); + + runInvalidQuery(statement, "SELECT * FROM test_table", "does not exist"); + } + + try (Statement statement = connection.createStatement()) { + // Objects outside "su_db" owned by "su" were not dropped. + statement.execute("SELECT * FROM test_table"); + + statement.execute("DROP OWNED BY su CASCADE"); + + runInvalidQuery(statement, "SELECT * FROM test_table", "does not exist"); + } + + // Database was not dropped, so we can reconnect. + try (Connection ignored = newConnectionBuilder().setDatabase("su_db").setUser("su").connect()) { + // No-op. + } + } + + @Test + public void testMultiDatabaseOwnershipReassign() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE su LOGIN SUPERUSER"); + statement.execute("CREATE ROLE recipient"); + + withRole(statement, "su", () -> { + // Create a database owned by "su". + statement.execute("CREATE DATABASE su_db"); + + // Create object in default database. + statement.execute("CREATE TABLE test_table(id int)"); + }); + } + + try (Connection connection = newConnectionBuilder().setDatabase("su_db") + .setUser("su").connect(); + Statement statement = connection.createStatement()) { + // Create object in "su_db". + statement.execute("CREATE TABLE test_table(id int)"); + + statement.execute("REASSIGN OWNED BY su TO recipient"); + + withRole(statement, "recipient", () -> + statement.execute("SELECT * FROM test_table")); + } + + try (Statement statement = connection.createStatement()) { + // Objects outside "su_db" owned by "su" were not reassigned. + withRole(statement, "recipient", () -> + runInvalidQuery(statement, "SELECT * FROM test_table", PERMISSION_DENIED)); + + statement.execute("REASSIGN OWNED BY su TO recipient"); + + withRole(statement, "recipient", () -> + statement.execute("SELECT * FROM test_table")); + } + } + + @Test + public void testCreateSchemaAuthorization() throws Exception { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE ROLE unprivileged"); + statement.execute("CREATE ROLE owner_role"); + + // Create schema and make "owner_role" the owner. + statement.execute("CREATE SCHEMA test_schema AUTHORIZATION owner_role"); + + withRole(statement, "owner_role", () -> { + // Owner has CREATE and USAGE privileges. + statement.execute("CREATE TABLE test_schema.test_table(id int)"); + statement.execute("SELECT * FROM test_schema.test_table"); + statement.execute("DROP TABLE test_schema.test_table"); + }); + + withRole(statement, "unprivileged", () -> { + // Other users have no privileges. + runInvalidQuery( + statement, + "CREATE TABLE test_schema.test_table2(id int)", + PERMISSION_DENIED + ); + runInvalidQuery(statement, "SELECT * FROM test_schema.test_table", PERMISSION_DENIED); + }); + + withRole(statement, "owner_role", () -> { + // Owner has DROP and GRANT privileges (owner-only privileges). + statement.execute("GRANT USAGE ON SCHEMA test_schema TO unprivileged"); + statement.execute("DROP SCHEMA test_schema"); + }); + } + } + + @Test + public void testRevokeLoginMidSession() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + + statement1.execute("CREATE ROLE test_role LOGIN"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + // Revoke login privileges now that "test_role" has connected. + statement1.execute("ALTER ROLE test_role NOLOGIN"); + + waitForTServerHeartbeat(); + + // Open session is still usable, even after the alter. + statement2.execute("SELECT count(*) FROM pg_class"); + } + + // Next login attempt fails. + try (Connection ignored = newConnectionBuilder().setUser("test_role").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("role \"test_role\" is not permitted to log in") + ); + } + } + } + + @Test + public void testGrantAttributesMidSession() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + + statement1.execute("CREATE ROLE test_role LOGIN"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + runInvalidQuery(statement2, "CREATE ROLE tr", PERMISSION_DENIED); + runInvalidQuery(statement2, "CREATE DATABASE tdb", PERMISSION_DENIED); + runInvalidQuery(statement2, "CREATE ROLE su SUPERUSER", "must be superuser"); + + // Grant CREATEROLE from connection 1. + statement1.execute("ALTER ROLE test_role CREATEROLE"); + + waitForTServerHeartbeat(); + + // New attribute observed on connection 2 after heartbeat. + statement2.execute("CREATE ROLE tr"); + runInvalidQuery(statement2, "CREATE DATABASE tdb", PERMISSION_DENIED); + runInvalidQuery(statement2, "CREATE ROLE su SUPERUSER", "must be superuser"); + + // Grant CREATEDB from connection 1. + statement1.execute("ALTER ROLE test_role CREATEDB"); + + waitForTServerHeartbeat(); + + // New attribute observed on connection 2 after heartbeat. + statement2.execute("CREATE DATABASE tdb"); + runInvalidQuery(statement2, "CREATE ROLE su SUPERUSER", "must be superuser"); + + // Grant SUPERUSER from connection 1. + statement1.execute("ALTER ROLE test_role SUPERUSER"); + + waitForTServerHeartbeat(); + + // New attribute observed on connection 2 after heartbeat. + statement2.execute("CREATE ROLE su SUPERUSER"); + + // "test_role" still cannot set their session authorization, despite having + // superuser privileges. + runInvalidQuery(statement2, "SET SESSION AUTHORIZATION su", PERMISSION_DENIED); + } + } + } + + @Test + public void testRevokeAttributesMidSession() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + + statement1.execute("CREATE ROLE test_role LOGIN CREATEROLE CREATEDB SUPERUSER"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + statement2.execute("CREATE ROLE su SUPERUSER"); + + // Revoke SUPERUSER from connection 1. + statement1.execute("ALTER ROLE test_role NOSUPERUSER"); + + waitForTServerHeartbeat(); + + // Lost attribute is observed from connection 2. + runInvalidQuery(statement2, "CREATE ROLE su SUPERUSER", "must be superuser"); + statement2.execute("CREATE DATABASE tdb"); + + // Revoke CREATEDB from connection 1. + statement1.execute("ALTER ROLE test_role NOCREATEDB"); + + waitForTServerHeartbeat(); + + // Lost attribute is observed from connection 2. + runInvalidQuery(statement2, "CREATE ROLE su SUPERUSER", "must be superuser"); + runInvalidQuery(statement2, "CREATE DATABASE tdb", PERMISSION_DENIED); + statement2.execute("CREATE ROLE tr"); + + // Revoke CREATEROLE from connection 1. + statement1.execute("ALTER ROLE test_role NOCREATEROLE"); + + waitForTServerHeartbeat(); + + // Lost attribute is observed from connection 2. + runInvalidQuery(statement2, "CREATE ROLE su SUPERUSER", "must be superuser"); + runInvalidQuery(statement2, "CREATE DATABASE tdb", PERMISSION_DENIED); + runInvalidQuery(statement2, "CREATE ROLE tr", PERMISSION_DENIED); + } + } + } + + @Test + public void testConnectionLimitDecreasedMidSession() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + + statement1.execute("CREATE ROLE test_role LOGIN CONNECTION LIMIT 1"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + // Decrease connection limit now that "test_role" has connected. + statement1.execute("ALTER ROLE test_role CONNECTION LIMIT 0"); + + waitForTServerHeartbeat(); + + // Open session is still usable, even after the alter. + statement2.execute("SELECT count(*) FROM pg_class"); + } + + // Next login attempt fails. + try (Connection ignored = newConnectionBuilder().setUser("test_role").connect()) { + fail("Expected login attempt to fail"); + } catch (SQLException sqle) { + assertThat( + sqle.getMessage(), + CoreMatchers.containsString("too many connections for role \"test_role\"") + ); + } + } + } + + @Test + public void testParentAttributeChangedMidSession() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + statement1.execute("CREATE ROLE test_role INHERIT LOGIN"); + statement1.execute("CREATE ROLE test_group INHERIT ROLE test_role"); + + // Create group with some privileges. + statement1.execute("CREATE ROLE test_large_group ROLE test_group"); + statement1.execute("CREATE TABLE test_table(id int)"); + statement1.execute("GRANT SELECT ON test_table TO test_large_group"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + // Connection 2 initially has select privileges. + statement2.execute("SELECT * FROM test_table"); + + // Disable parent inheritance from connection 1. + statement1.execute("ALTER ROLE test_group NOINHERIT"); + + waitForTServerHeartbeat(); + + // Connection 2 has lost inherited privileges. + runInvalidQuery(statement2, "SELECT * FROM test_table", PERMISSION_DENIED); + + // Enable parent inheritance from connection 1. + statement1.execute("ALTER ROLE test_group INHERIT"); + + waitForTServerHeartbeat(); + + // Select privileges restored to connection 2. + statement2.execute("SELECT * FROM test_table"); + } + } + } + + @Test + public void testMembershipRevokedInsideGroup() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + statement1.execute("CREATE ROLE test_role LOGIN"); + + // Create group with some privileges. + statement1.execute("CREATE ROLE test_group ROLE test_role"); + statement1.execute("CREATE TABLE test_table(id int)"); + statement1.execute("GRANT SELECT ON test_table to test_group"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + statement2.execute("SET ROLE test_group"); + + // Can initially use group privileges. + assertQuery(statement2, "SELECT * FROM test_table"); + + // Revoke membership on another node. + statement1.execute("REVOKE test_group FROM test_role"); + + waitForTServerHeartbeat(); + + // Can still use group privileges, even though membership has been revoked. + assertQuery(statement2, "SELECT * FROM test_table"); + + // Cannot set role to the group anymore. + runInvalidQuery(statement2, "SET ROLE test_group", PERMISSION_DENIED); + statement2.execute("RESET ROLE"); + runInvalidQuery(statement2, "SET ROLE test_group", PERMISSION_DENIED); + } + } + } + + @Test + public void testMembershipRevokedOutsideGroup() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + statement1.execute("CREATE ROLE test_role LOGIN INHERIT"); + + // Create group with some privileges. + statement1.execute("CREATE ROLE test_group ROLE test_role"); + statement1.execute("CREATE TABLE test_table(id int)"); + statement1.execute("GRANT SELECT ON test_table to test_group"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + // Can initially use group privileges. + assertQuery(statement2, "SELECT * FROM test_table"); + + // Revoke membership on another node. + statement1.execute("REVOKE test_group FROM test_role"); + + waitForTServerHeartbeat(); + + // Immediately lose group privileges. + runInvalidQuery(statement2, "SELECT * FROM test_table", PERMISSION_DENIED); + + // Cannot set role to the group anymore. + runInvalidQuery(statement2, "SET ROLE test_group", PERMISSION_DENIED); + } + } + } + + @Test + public void testInheritanceSwitchedWhileInsideGroup() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + statement1.execute("CREATE ROLE test_role LOGIN INHERIT"); + + // Create group with some privileges. + statement1.execute("CREATE ROLE test_group ROLE test_role"); + statement1.execute("CREATE TABLE test_table(id int)"); + statement1.execute("GRANT SELECT ON test_table to test_group"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + // Connection 2 initially has select privileges. + statement2.execute("SELECT * FROM test_table"); + + // Remove inheritance from connection 1. + statement1.execute("ALTER ROLE test_role NOINHERIT"); + + waitForTServerHeartbeat(); + + // Connection 2 has lost inherited privileges. + runInvalidQuery(statement2, "SELECT * FROM test_table", PERMISSION_DENIED); + + // Add inheritance from connection 1. + statement1.execute("ALTER ROLE test_role INHERIT"); + + waitForTServerHeartbeat(); + + // Select privileges restored to connection 2. + statement2.execute("SELECT * FROM test_table"); + } + } + } + + @Test + public void testRoleRenamingMidSession() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Statement statement1 = connection1.createStatement()) { + + statement1.execute("CREATE ROLE test_role LOGIN"); + statement1.execute("CREATE GROUP test_group ROLE test_role"); + + try (Connection connection2 = newConnectionBuilder().setTServer(1) + .setUser("test_role").connect(); + Statement statement2 = connection2.createStatement()) { + statement2.execute("SET ROLE test_group"); + + assertEquals("test_role", getSessionUser(statement2)); + assertEquals("test_group", getCurrentUser(statement2)); + + // Rename role from connection 1. + statement1.execute("ALTER ROLE test_group RENAME TO test_group1"); + + waitForTServerHeartbeat(); + + // New name is visible from connection 2. + assertEquals("test_role", getSessionUser(statement2)); + assertEquals("test_group1", getCurrentUser(statement2)); + + // Rename role from connection 1. + statement1.execute("ALTER ROLE test_role RENAME TO test_role1"); + + waitForTServerHeartbeat(); + + // New name is visible from connection 2. + assertEquals("test_role1", getSessionUser(statement2)); + assertEquals("test_group1", getCurrentUser(statement2)); + } + } + } + + @Test + public void testMultiNodePermissionChanges() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { + statement1.execute("CREATE TABLE test_table(id int)"); + + statement2.execute("CREATE ROLE test_role"); + + withRole(statement2, "test_role", () -> { + // No permissions by default for table. + runInvalidQuery(statement2, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement2, "INSERT INTO test_table VALUES (1)", PERMISSION_DENIED); + + // Grant SELECT from connection 1. + statement1.execute("GRANT SELECT ON TABLE test_table TO test_role"); + + waitForTServerHeartbeat(); + + statement2.execute("SELECT * FROM test_table"); + runInvalidQuery(statement2, "INSERT INTO test_table VALUES (1)", PERMISSION_DENIED); + + // Grant INSERT from connection 1. + statement1.execute("GRANT INSERT ON TABLE test_table TO test_role"); + + waitForTServerHeartbeat(); + + statement2.execute("SELECT * FROM test_table"); + statement2.execute("INSERT INTO test_table VALUES (1)"); + + // Revoke all from connection 1. + statement1.execute("REVOKE ALL ON TABLE test_table FROM test_role"); + + waitForTServerHeartbeat(); + + runInvalidQuery(statement2, "SELECT * FROM test_table", PERMISSION_DENIED); + runInvalidQuery(statement2, "INSERT INTO test_table VALUES (1)", PERMISSION_DENIED); + }); + } + } + + @Test + public void testAlterDefaultPrivilegesAcrossNodes() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { + statement1.execute("CREATE SCHEMA ts"); + statement1.execute("GRANT USAGE ON SCHEMA ts TO PUBLIC"); + statement1.execute("CREATE ROLE test_role LOGIN"); + statement2.execute("CREATE TABLE ts.table1(id int)"); + + try (Connection connection3 = newConnectionBuilder().setTServer(2) + .setUser("test_role").connect(); + Statement statement3 = connection3.createStatement()) { + runInvalidQuery(statement3, "SELECT * FROM ts.table1", PERMISSION_DENIED); + + // Grant select by default from connection 1. + statement1.execute( + "ALTER DEFAULT PRIVILEGES IN SCHEMA ts GRANT SELECT ON TABLES TO PUBLIC"); + waitForTServerHeartbeat(); + statement2.execute("CREATE TABLE ts.table2(id int)"); + + // New table has select privileges, old tables unaffected. + runInvalidQuery(statement3, "SELECT * FROM ts.table1", PERMISSION_DENIED); + statement3.execute("SELECT * FROM ts.table2"); + runInvalidQuery(statement3, "INSERT INTO ts.table2(id) VALUES (1)", PERMISSION_DENIED); + + // Grant insert by default from connection 1. + statement1.execute( + "ALTER DEFAULT PRIVILEGES IN SCHEMA ts GRANT INSERT ON TABLES TO PUBLIC"); + waitForTServerHeartbeat(); + statement2.execute("CREATE TABLE ts.table3(id int)"); + + // New table has select and insert privileges, old tables unaffected. + runInvalidQuery(statement3, "SELECT * FROM ts.table1", PERMISSION_DENIED); + statement3.execute("SELECT * FROM ts.table2"); + runInvalidQuery(statement3, "INSERT INTO ts.table2(id) VALUES (1)", PERMISSION_DENIED); + statement3.execute("SELECT * FROM ts.table3"); + statement3.execute("INSERT INTO ts.table3(id) VALUES (1)"); + + // Revoke all privileges by default from connection 1. + statement1.execute( + "ALTER DEFAULT PRIVILEGES IN SCHEMA ts REVOKE ALL ON TABLES FROM PUBLIC"); + waitForTServerHeartbeat(); + statement2.execute("CREATE TABLE ts.table4(id int)"); + + // New table has no privileges, old tables unaffected. + runInvalidQuery(statement3, "SELECT * FROM ts.table1", PERMISSION_DENIED); + statement3.execute("SELECT * FROM ts.table2"); + runInvalidQuery(statement3, "INSERT INTO ts.table2(id) VALUES (1)", PERMISSION_DENIED); + statement3.execute("SELECT * FROM ts.table3"); + statement3.execute("INSERT INTO ts.table3(id) VALUES (1)"); + runInvalidQuery(statement3, "SELECT * FROM ts.table4", PERMISSION_DENIED); + runInvalidQuery(statement3, "INSERT INTO ts.table4(id) VALUES (1)", PERMISSION_DENIED); + } + } + } + + + @Test + public void testMultiNodeOwnershipChanges() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { + statement1.execute("CREATE ROLE role1"); + statement2.execute("CREATE ROLE role2"); + + waitForTServerHeartbeat(); + + withRole(statement2, "role2", () -> { + withRole(statement1, "role1", () -> { + // Create an object owned by role1. + statement1.execute("CREATE TABLE test_table(id int)"); + }); + + // Fill cache with existing privileges. + runInvalidQuery(statement2, "SELECT * FROM test_table", PERMISSION_DENIED); + + // Reassign ownership to role2 from connection 1. + statement1.execute("REASSIGN OWNED BY role1 TO role2"); + + waitForTServerHeartbeat(); + + // Connection 2 observes the ownership change. + statement2.execute("ALTER TABLE test_table ADD COLUMN a int"); + + // Drop owned objects from connection 1. + statement1.execute("DROP OWNED BY role2"); + + waitForTServerHeartbeat(); + + // Connection 2 observes objects dropped. + runInvalidQuery( + statement2, + "ALTER TABLE test_table ADD COLUMN b int", + "relation \"test_table\" does not exist" + ); + + // Create table from connection 1. + statement1.execute("CREATE TABLE other_table(id int)"); + + // Fill cache with existing privileges. + runInvalidQuery(statement2, "SELECT * FROM other_table", PERMISSION_DENIED); + + statement1.execute("ALTER TABLE other_table OWNER TO role2"); + + waitForTServerHeartbeat(); + + // Connection 2 observers owner privileges. + statement2.execute("DROP TABLE other_table"); + }); + } + } + + interface ThrowingRunnable { + void run() throws Exception; + } + + private static void withRoles( + Statement statement, + Set roles, + ThrowingRunnable runnable + ) throws Exception { + for (String role : roles) { + withRole(statement, role, runnable); + } + } + + private static void withRole( + Statement statement, + String role, + ThrowingRunnable runnable + ) throws Exception { + String sessionUser = getSessionUser(statement); + + statement.execute(String.format("SET SESSION AUTHORIZATION %s", role)); + runnable.run(); + statement.execute(String.format("SET SESSION AUTHORIZATION %s", sessionUser)); + } + + private static class RoleSet extends TreeSet { + RoleSet(String... roles) { + super(); + addAll(Lists.newArrayList(roles)); + } + + RoleSet(Set roles) { + super(roles); + } + + RoleSet excluding(String... roles) { + RoleSet newSet = new RoleSet(this); + newSet.removeAll(Lists.newArrayList(roles)); + return newSet; + } + } + + private static String getSessionUser(Statement statement) throws Exception { + ResultSet resultSet = statement.executeQuery("SELECT SESSION_USER"); + resultSet.next(); + return resultSet.getString(1); + } + + private static String getCurrentUser(Statement statement) throws Exception { + ResultSet resultSet = statement.executeQuery("SELECT CURRENT_USER"); + resultSet.next(); + return resultSet.getString(1); + } +} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgCacheConsistency.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgCacheConsistency.java index e981fb2e3ed6..cea05b189654 100644 --- a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgCacheConsistency.java +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgCacheConsistency.java @@ -155,8 +155,10 @@ public void testNoDDLRetry() throws Exception { @Test public void testVersionMismatchWithoutRetry() throws Exception { - try (Statement statement1 = createConnection(0).createStatement(); - Statement statement2 = createConnection(1).createStatement()) { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { statement1.execute("CREATE TABLE test_table(id int, PRIMARY KEY (id))"); statement1.execute("INSERT INTO test_table(id) VALUES (1), (2), (3)"); @@ -208,8 +210,10 @@ public void testVersionMismatchWithoutRetry() throws Exception { @Test public void testVersionMismatchWithFailedRetry() throws Exception { - try (Statement statement1 = createConnection(0).createStatement(); - Statement statement2 = createConnection(1).createStatement()) { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { // Create table from connection 1. statement1.execute("CREATE TABLE test_table(id int)"); @@ -259,8 +263,10 @@ public void testVersionMismatchWithFailedRetry() throws Exception { @Ignore // TODO enable after #1502 public void testUndetectedSelectVersionMismatch() throws Exception { - try (Statement statement1 = createConnection(0).createStatement(); - Statement statement2 = createConnection(1).createStatement()) { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { // Create table from connection 1. statement1.execute("CREATE TABLE test_table(id int, PRIMARY KEY (id))"); @@ -280,8 +286,10 @@ public void testUndetectedSelectVersionMismatch() throws Exception { @Test public void testConsistentNonRetryableTransactions() throws Exception { - try (Statement statement1 = createConnection(0).createStatement(); - Statement statement2 = createConnection(1).createStatement()) { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { // Create table from connection 1. statement1.execute("CREATE TABLE test_table(id int, PRIMARY KEY (id))"); @@ -307,8 +315,10 @@ public void testConsistentNonRetryableTransactions() throws Exception { @Test public void testConsistentPreparedStatements() throws Exception { - try (Statement statement1 = createConnection(0).createStatement(); - Statement statement2 = createConnection(1).createStatement()) { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { // Create table from connection 1. statement1.execute("CREATE TABLE test_table(id int, PRIMARY KEY (id))"); @@ -349,8 +359,10 @@ public void testConsistentPreparedStatements() throws Exception { @Test public void testConsistentExplain() throws Exception { - try (Statement statement1 = createConnection(0).createStatement(); - Statement statement2 = createConnection(1).createStatement()) { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { // Create table with unique column from connection 1. statement1.execute("CREATE TABLE test_table(id int, u int)"); statement1.execute("ALTER TABLE test_table ADD CONSTRAINT unq UNIQUE (u)"); @@ -382,6 +394,51 @@ public void testConsistentExplain() throws Exception { } } + @Test + public void testConsistentGUCWrites() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { + statement1.execute("CREATE ROLE some_role"); + + // Update roles cache on connection 2. + statement2.execute("SET ROLE some_role"); + statement2.execute("RESET ROLE"); + + statement1.execute("DROP ROLE some_role"); + + waitForTServerHeartbeat(); + + // Connection 2 refreshes its cache before setting the guc var. + runInvalidQuery(statement2, "SET ROLE some_role", "role \"some_role\" does not exist"); + } + } + + @Test + public void testInvalidationCallbacksWhenInsertingIntoList() throws Exception { + try (Connection connection1 = newConnectionBuilder().setTServer(0).connect(); + Connection connection2 = newConnectionBuilder().setTServer(1).connect(); + Statement statement1 = connection1.createStatement(); + Statement statement2 = connection2.createStatement()) { + statement1.execute("CREATE ROLE some_role CREATEROLE"); + + statement2.execute("SET SESSION AUTHORIZATION some_role"); + + // Populate membership roles cache from connection 2. + statement2.execute("CREATE ROLE inaccessible"); + runInvalidQuery(statement2, "SET ROLE inaccessible", "permission denied"); + + // Invalidate membership roles cache from connection 1. + statement1.execute("CREATE ROLE some_group ROLE some_role"); + + waitForTServerHeartbeat(); + + // Connection 2 observes the new membership roles list. + statement2.execute("SET ROLE some_group"); + } + } + private interface ThrowingRunnable { void run() throws Throwable; } diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgRegressAuthorization.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgRegressAuthorization.java new file mode 100644 index 000000000000..7e8ead3a0765 --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/TestPgRegressAuthorization.java @@ -0,0 +1,33 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.yb.util.YBTestRunnerNonTsanOnly; + +/** + * Runs the pg_regress authorization-related tests on YB code. + */ +@RunWith(value = YBTestRunnerNonTsanOnly.class) +public class TestPgRegressAuthorization extends BasePgSQLTest { + @Override + public int getTestMethodTimeoutSec() { + return 1800; + } + + @Test + public void testPgRegressAuthorization() throws Exception { + runPgRegressTest("yb_pg_auth_serial_schedule"); + } +} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/ClusterCleaner.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/ClusterCleaner.java new file mode 100644 index 000000000000..a52fdce0007f --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/ClusterCleaner.java @@ -0,0 +1,24 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql.cleaners; + +import java.sql.Connection; + +/** + * Interface representing an object which can perform some post-test cleanup + * on a postgres cluster. + */ +public interface ClusterCleaner { + void clean(Connection connection) throws Exception; +} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/ConnectionCleaner.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/ConnectionCleaner.java new file mode 100644 index 000000000000..00dd1af3f14b --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/ConnectionCleaner.java @@ -0,0 +1,82 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql.cleaners; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; + +/** + * Closes all registered postgres connections, except the connection passed + * to {@link ConnectionCleaner#clean(Connection)}. + */ +public class ConnectionCleaner implements ClusterCleaner { + private static final Logger LOG = LoggerFactory.getLogger(ConnectionCleaner.class); + + private static List connectionsToClose = new ArrayList<>(); + + public static void register(Connection connection) { + connectionsToClose.add(connection); + } + + @Override + public void clean(Connection rootConnection) throws Exception { + LOG.info("Cleaning-up postgres connections"); + + if (rootConnection != null) { + try (Statement statement = rootConnection.createStatement()) { + try (ResultSet resultSet = statement.executeQuery( + "SELECT client_hostname, client_port, state, query, pid FROM pg_stat_activity")) { + while (resultSet.next()) { + int backendPid = resultSet.getInt(5); + LOG.info(String.format( + "Found connection: hostname=%s, port=%s, state=%s, query=%s, backend_pid=%s", + resultSet.getString(1), resultSet.getInt(2), + resultSet.getString(3), resultSet.getString(4), backendPid)); + } + } + } catch (SQLException e) { + LOG.info("Exception when trying to list PostgreSQL connections", e); + } + + LOG.info("Closing connections."); + for (Connection connection : connectionsToClose) { + // Keep the main connection alive between tests. + if (connection == rootConnection) continue; + + try { + if (connection == null) { + LOG.error("connectionsToClose contains a null connection!"); + } else { + connection.close(); + } + } catch (SQLException ex) { + LOG.error("Exception while trying to close connection"); + throw ex; + } + } + } else { + LOG.info("Connection is already null, nothing to close"); + } + LOG.info("Finished closing connection."); + + connectionsToClose.clear(); + } +} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/DatabaseCleaner.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/DatabaseCleaner.java new file mode 100644 index 000000000000..0805ef9ac770 --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/DatabaseCleaner.java @@ -0,0 +1,57 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql.cleaners; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; + +/** + * Removes all databases excluding `postgres`, `template1`, and `template2`. + * Any lower-priority cleaners should only clean objects in one of the remaining + * three databases, or cluster-wide objects (e.g. roles). + * The passed connection must be open in one of the three databases listed above. + */ +public class DatabaseCleaner implements ClusterCleaner { + private static final Logger LOG = LoggerFactory.getLogger(DatabaseCleaner.class); + + @Override + public void clean(Connection connection) throws Exception { + LOG.info("Cleaning-up non-standard postgres databases"); + + try (Statement statement = connection.createStatement()) { + statement.execute("RESET SESSION AUTHORIZATION"); + + ResultSet resultSet = statement.executeQuery( + "SELECT datname FROM pg_database" + + " WHERE datname <> 'template0'" + + " AND datname <> 'template1'" + + " AND datname <> 'postgres'"); + + List databases = new ArrayList<>(); + while (resultSet.next()) { + databases.add(resultSet.getString(1)); + } + + for (String database : databases) { + statement.execute("DROP DATABASE " + database); + } + } + } +} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/RoleCleaner.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/RoleCleaner.java new file mode 100644 index 000000000000..678af70af8d5 --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/RoleCleaner.java @@ -0,0 +1,59 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql.cleaners; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yb.pgsql.BasePgSQLTest; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; + +/** + * Removes all non-standard roles (except the test role), as well as their permissions and + * owned objects. + * NOTE: This will fail if any roles own objects in databases other than `postgres`, + * so {@link DatabaseCleaner} must be run first. + */ +public class RoleCleaner implements ClusterCleaner { + private static final Logger LOG = LoggerFactory.getLogger(RoleCleaner.class); + + @Override + public void clean(Connection connection) throws Exception { + LOG.info("Cleaning-up postgres roles and permissions"); + + try (Statement statement = connection.createStatement()) { + statement.execute("RESET SESSION AUTHORIZATION"); + + ResultSet resultSet = statement.executeQuery( + "SELECT rolname FROM pg_roles" + + " WHERE rolname <> 'postgres'" + + " AND rolname <> '" + BasePgSQLTest.TEST_PG_USER + "'" + + " AND rolname NOT LIKE 'pg_%'"); + + List roles = new ArrayList<>(); + while (resultSet.next()) { + roles.add(resultSet.getString(1)); + } + + for (String role : roles) { + statement.execute("DROP OWNED BY " + role + " CASCADE"); + statement.execute("DROP ROLE " + role); + } + } + } +} diff --git a/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/UserObjectCleaner.java b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/UserObjectCleaner.java new file mode 100644 index 000000000000..861bb7a55c2a --- /dev/null +++ b/java/yb-pgsql/src/test/java/org/yb/pgsql/cleaners/UserObjectCleaner.java @@ -0,0 +1,38 @@ +// Copyright (c) YugaByte, Inc. +// +// 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 org.yb.pgsql.cleaners; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yb.pgsql.BasePgSQLTest; + +import java.sql.Connection; +import java.sql.Statement; + +/** + * Removes all objects owned by the test user from the database of the passed connection. + */ +public class UserObjectCleaner implements ClusterCleaner { + private static final Logger LOG = LoggerFactory.getLogger(UserObjectCleaner.class); + + @Override + public void clean(Connection connection) throws Exception { + LOG.info("Cleaning-up postgres test user objects"); + + try (Statement statement = connection.createStatement()) { + statement.execute("RESET SESSION AUTHORIZATION"); + statement.execute("DROP OWNED BY " + BasePgSQLTest.TEST_PG_USER + " CASCADE"); + } + } +} diff --git a/src/postgres/src/backend/access/ybc/ybcam.c b/src/postgres/src/backend/access/ybc/ybcam.c index d24e20c81c1f..d8a4e9ade870 100644 --- a/src/postgres/src/backend/access/ybc/ybcam.c +++ b/src/postgres/src/backend/access/ybc/ybcam.c @@ -248,6 +248,10 @@ static HeapTuple ybcFetchNextHeapTuple(YbScanDesc ybScan, bool is_forward_scan) { tuple->t_ybctid = PointerGetDatum(syscols.ybctid); } + if (ybScan->tableOid != InvalidOid) + { + tuple->t_tableOid = ybScan->tableOid; + } } pfree(values); pfree(nulls); @@ -453,6 +457,7 @@ ybcBeginScan(Relation relation, Relation index, bool index_cols_only, int nkeys, ybScan->key = key; ybScan->nkeys = nkeys; ybScan->exec_params = NULL; + ybScan->tableOid = RelationGetRelid(relation); /* Setup up the scan plan */ YbScanPlanData scan_plan; @@ -1196,6 +1201,7 @@ HeapTuple YBCFetchTuple(Relation relation, Datum ybctid) { tuple->t_ybctid = PointerGetDatum(syscols.ybctid); } + tuple->t_tableOid = RelationGetRelid(relation); } pfree(values); pfree(nulls); diff --git a/src/postgres/src/backend/catalog/aclchk.c b/src/postgres/src/backend/catalog/aclchk.c index c4ace85d65e1..abc20a900192 100644 --- a/src/postgres/src/backend/catalog/aclchk.c +++ b/src/postgres/src/backend/catalog/aclchk.c @@ -5467,10 +5467,6 @@ get_user_default_acl(ObjectType objtype, Oid ownerId, Oid nsp_oid) if (IsBootstrapProcessingMode()) return NULL; - /* TODO Do not support default permissions in YugaByte yet. */ - if (IsYugaByteEnabled()) - return NULL; - /* Check if object type is supported in pg_default_acl */ switch (objtype) { diff --git a/src/postgres/src/backend/catalog/pg_constraint.c b/src/postgres/src/backend/catalog/pg_constraint.c index 920d1a9bf6f0..f0726e9aa06f 100644 --- a/src/postgres/src/backend/catalog/pg_constraint.c +++ b/src/postgres/src/backend/catalog/pg_constraint.c @@ -36,6 +36,7 @@ #include "utils/rel.h" #include "utils/syscache.h" #include "utils/tqual.h" +#include "pg_yb_utils.h" /* @@ -892,7 +893,7 @@ get_relation_constraint_oid(Oid relid, const char *conname, bool missing_ok) * failure. */ Bitmapset * -get_relation_constraint_attnos(Oid relid, const char *conname, +get_relation_constraint_attnos(Relation rel, const char *conname, bool missing_ok, Oid *constraintOid) { Bitmapset *conattnos = NULL; @@ -900,6 +901,7 @@ get_relation_constraint_attnos(Oid relid, const char *conname, HeapTuple tuple; SysScanDesc scan; ScanKeyData skey[3]; + Oid relid = RelationGetRelid(rel); /* Set *constraintOid, to avoid complaints about uninitialized vars */ *constraintOid = InvalidOid; @@ -953,7 +955,7 @@ get_relation_constraint_attnos(Oid relid, const char *conname, for (i = 0; i < numcols; i++) { conattnos = bms_add_member(conattnos, - attnums[i] - FirstLowInvalidHeapAttributeNumber); + attnums[i] - YBGetFirstLowInvalidAttributeNumber(rel)); } } } diff --git a/src/postgres/src/backend/catalog/pg_shdepend.c b/src/postgres/src/backend/catalog/pg_shdepend.c index dece489dc868..7ac29a89dfec 100644 --- a/src/postgres/src/backend/catalog/pg_shdepend.c +++ b/src/postgres/src/backend/catalog/pg_shdepend.c @@ -132,16 +132,6 @@ recordSharedDependencyOn(ObjectAddress *depender, if (IsBootstrapProcessingMode()) return; - /* - * Disable dependency check here for now to avoid many full-table - * scans caused by isObjectPinned below. YugaByte master catalog maps - * should catch broken dependencies for now anyway. - * TODO as we enable more postgres-exclusive features this needs to be - * handled (re-enabled) to ensure correctness. - */ - if (IsYugaByteEnabled()) - return; - sdepRel = heap_open(SharedDependRelationId, RowExclusiveLock); /* If the referenced object is pinned, do nothing. */ diff --git a/src/postgres/src/backend/catalog/system_views.sql b/src/postgres/src/backend/catalog/system_views.sql index 046293d20fd1..8cd8bf40ac49 100644 --- a/src/postgres/src/backend/catalog/system_views.sql +++ b/src/postgres/src/backend/catalog/system_views.sql @@ -939,6 +939,238 @@ GRANT SELECT (subdbid, subname, subowner, subenabled, subslotname, subpublicatio ON pg_subscription TO public; +-- +-- We have a few function definitions in here, too. +-- At some point there might be enough to justify breaking them out into +-- a separate "system_functions.sql" file. +-- + +-- Tsearch debug function. Defined here because it'd be pretty unwieldy +-- to put it into pg_proc.h + +CREATE FUNCTION ts_debug(IN config regconfig, IN document text, + OUT alias text, + OUT description text, + OUT token text, + OUT dictionaries regdictionary[], + OUT dictionary regdictionary, + OUT lexemes text[]) +RETURNS SETOF record AS +$$ +SELECT + tt.alias AS alias, + tt.description AS description, + parse.token AS token, + ARRAY ( SELECT m.mapdict::pg_catalog.regdictionary + FROM pg_catalog.pg_ts_config_map AS m + WHERE m.mapcfg = $1 AND m.maptokentype = parse.tokid + ORDER BY m.mapseqno ) + AS dictionaries, + ( SELECT mapdict::pg_catalog.regdictionary + FROM pg_catalog.pg_ts_config_map AS m + WHERE m.mapcfg = $1 AND m.maptokentype = parse.tokid + ORDER BY pg_catalog.ts_lexize(mapdict, parse.token) IS NULL, m.mapseqno + LIMIT 1 + ) AS dictionary, + ( SELECT pg_catalog.ts_lexize(mapdict, parse.token) + FROM pg_catalog.pg_ts_config_map AS m + WHERE m.mapcfg = $1 AND m.maptokentype = parse.tokid + ORDER BY pg_catalog.ts_lexize(mapdict, parse.token) IS NULL, m.mapseqno + LIMIT 1 + ) AS lexemes +FROM pg_catalog.ts_parse( + (SELECT cfgparser FROM pg_catalog.pg_ts_config WHERE oid = $1 ), $2 + ) AS parse, + pg_catalog.ts_token_type( + (SELECT cfgparser FROM pg_catalog.pg_ts_config WHERE oid = $1 ) + ) AS tt +WHERE tt.tokid = parse.tokid +$$ +LANGUAGE SQL STRICT STABLE PARALLEL SAFE; + +COMMENT ON FUNCTION ts_debug(regconfig,text) IS + 'debug function for text search configuration'; + +CREATE FUNCTION ts_debug(IN document text, + OUT alias text, + OUT description text, + OUT token text, + OUT dictionaries regdictionary[], + OUT dictionary regdictionary, + OUT lexemes text[]) +RETURNS SETOF record AS +$$ + SELECT * FROM pg_catalog.ts_debug( pg_catalog.get_current_ts_config(), $1); +$$ +LANGUAGE SQL STRICT STABLE PARALLEL SAFE; + +COMMENT ON FUNCTION ts_debug(text) IS + 'debug function for current text search configuration'; + +-- +-- Redeclare built-in functions that need default values attached to their +-- arguments. It's impractical to set those up directly in pg_proc.h because +-- of the complexity and platform-dependency of the expression tree +-- representation. (Note that internal functions still have to have entries +-- in pg_proc.h; we are merely causing their proargnames and proargdefaults +-- to get filled in.) +-- + +CREATE OR REPLACE FUNCTION + pg_start_backup(label text, fast boolean DEFAULT false, exclusive boolean DEFAULT true) + RETURNS pg_lsn STRICT VOLATILE LANGUAGE internal AS 'pg_start_backup' + PARALLEL RESTRICTED; + +CREATE OR REPLACE FUNCTION pg_stop_backup ( + exclusive boolean, wait_for_archive boolean DEFAULT true, + OUT lsn pg_lsn, OUT labelfile text, OUT spcmapfile text) + RETURNS SETOF record STRICT VOLATILE LANGUAGE internal as 'pg_stop_backup_v2' + PARALLEL RESTRICTED; + +-- legacy definition for compatibility with 9.3 +CREATE OR REPLACE FUNCTION + json_populate_record(base anyelement, from_json json, use_json_as_text boolean DEFAULT false) + RETURNS anyelement LANGUAGE internal STABLE AS 'json_populate_record' PARALLEL SAFE; + +-- legacy definition for compatibility with 9.3 +CREATE OR REPLACE FUNCTION + json_populate_recordset(base anyelement, from_json json, use_json_as_text boolean DEFAULT false) + RETURNS SETOF anyelement LANGUAGE internal STABLE ROWS 100 AS 'json_populate_recordset' PARALLEL SAFE; + +CREATE OR REPLACE FUNCTION pg_logical_slot_get_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data text) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_get_changes'; + +CREATE OR REPLACE FUNCTION pg_logical_slot_peek_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data text) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_peek_changes'; + +CREATE OR REPLACE FUNCTION pg_logical_slot_get_binary_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data bytea) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_get_binary_changes'; + +CREATE OR REPLACE FUNCTION pg_logical_slot_peek_binary_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data bytea) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_peek_binary_changes'; + +CREATE OR REPLACE FUNCTION pg_create_physical_replication_slot( + IN slot_name name, IN immediately_reserve boolean DEFAULT false, + IN temporary boolean DEFAULT false, + OUT slot_name name, OUT lsn pg_lsn) +RETURNS RECORD +LANGUAGE INTERNAL +STRICT VOLATILE +AS 'pg_create_physical_replication_slot'; + +CREATE OR REPLACE FUNCTION pg_create_logical_replication_slot( + IN slot_name name, IN plugin name, + IN temporary boolean DEFAULT false, + OUT slot_name text, OUT lsn pg_lsn) +RETURNS RECORD +LANGUAGE INTERNAL +STRICT VOLATILE +AS 'pg_create_logical_replication_slot'; + +CREATE OR REPLACE FUNCTION + make_interval(years int4 DEFAULT 0, months int4 DEFAULT 0, weeks int4 DEFAULT 0, + days int4 DEFAULT 0, hours int4 DEFAULT 0, mins int4 DEFAULT 0, + secs double precision DEFAULT 0.0) +RETURNS interval +LANGUAGE INTERNAL +STRICT IMMUTABLE PARALLEL SAFE +AS 'make_interval'; + +CREATE OR REPLACE FUNCTION + jsonb_set(jsonb_in jsonb, path text[] , replacement jsonb, + create_if_missing boolean DEFAULT true) +RETURNS jsonb +LANGUAGE INTERNAL +STRICT IMMUTABLE PARALLEL SAFE +AS 'jsonb_set'; + +CREATE OR REPLACE FUNCTION + parse_ident(str text, strict boolean DEFAULT true) +RETURNS text[] +LANGUAGE INTERNAL +STRICT IMMUTABLE PARALLEL SAFE +AS 'parse_ident'; + +CREATE OR REPLACE FUNCTION + jsonb_insert(jsonb_in jsonb, path text[] , replacement jsonb, + insert_after boolean DEFAULT false) +RETURNS jsonb +LANGUAGE INTERNAL +STRICT IMMUTABLE PARALLEL SAFE +AS 'jsonb_insert'; + +-- +-- The default permissions for functions mean that anyone can execute them. +-- A number of functions shouldn't be executable by just anyone, but rather +-- than use explicit 'superuser()' checks in those functions, we use the GRANT +-- system to REVOKE access to those functions at initdb time. Administrators +-- can later change who can access these functions, or leave them as only +-- available to superuser / cluster owner, if they choose. +-- +REVOKE EXECUTE ON FUNCTION pg_start_backup(text, boolean, boolean) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stop_backup() FROM public; +REVOKE EXECUTE ON FUNCTION pg_stop_backup(boolean, boolean) FROM public; +REVOKE EXECUTE ON FUNCTION pg_create_restore_point(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_switch_wal() FROM public; +REVOKE EXECUTE ON FUNCTION pg_wal_replay_pause() FROM public; +REVOKE EXECUTE ON FUNCTION pg_wal_replay_resume() FROM public; +REVOKE EXECUTE ON FUNCTION pg_rotate_logfile() FROM public; +REVOKE EXECUTE ON FUNCTION pg_reload_conf() FROM public; +REVOKE EXECUTE ON FUNCTION pg_current_logfile() FROM public; +REVOKE EXECUTE ON FUNCTION pg_current_logfile(text) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_stat_reset() FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_reset_shared(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_reset_single_table_counters(oid) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_reset_single_function_counters(oid) FROM public; + +REVOKE EXECUTE ON FUNCTION lo_import(text) FROM public; +REVOKE EXECUTE ON FUNCTION lo_import(text, oid) FROM public; +REVOKE EXECUTE ON FUNCTION lo_export(oid, text) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_ls_logdir() FROM public; +REVOKE EXECUTE ON FUNCTION pg_ls_waldir() FROM public; + +REVOKE EXECUTE ON FUNCTION pg_read_file(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_file(text,bigint,bigint) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_file(text,bigint,bigint,boolean) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_read_binary_file(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_binary_file(text,bigint,bigint) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_binary_file(text,bigint,bigint,boolean) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_stat_file(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_file(text,boolean) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_ls_dir(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_ls_dir(text,boolean,boolean) FROM public; + +-- +-- We also set up some things as accessible to standard roles. +-- +GRANT EXECUTE ON FUNCTION pg_ls_logdir() TO pg_monitor; +GRANT EXECUTE ON FUNCTION pg_ls_waldir() TO pg_monitor; + GRANT pg_read_all_settings TO pg_monitor; GRANT pg_read_all_stats TO pg_monitor; GRANT pg_stat_scan_tables TO pg_monitor; diff --git a/src/postgres/src/backend/catalog/yb_system_views.sql b/src/postgres/src/backend/catalog/yb_system_views.sql index 8acaa921025d..8cd8bf40ac49 100644 --- a/src/postgres/src/backend/catalog/yb_system_views.sql +++ b/src/postgres/src/backend/catalog/yb_system_views.sql @@ -938,8 +938,74 @@ REVOKE ALL ON pg_subscription FROM public; GRANT SELECT (subdbid, subname, subowner, subenabled, subslotname, subpublications) ON pg_subscription TO public; --- YB NOTE : We have removed a bunch of unsupported / unnecessary CREATE FUNCTION --- declarations and related grant/revoke commands (compared to standard system_views.sql). + +-- +-- We have a few function definitions in here, too. +-- At some point there might be enough to justify breaking them out into +-- a separate "system_functions.sql" file. +-- + +-- Tsearch debug function. Defined here because it'd be pretty unwieldy +-- to put it into pg_proc.h + +CREATE FUNCTION ts_debug(IN config regconfig, IN document text, + OUT alias text, + OUT description text, + OUT token text, + OUT dictionaries regdictionary[], + OUT dictionary regdictionary, + OUT lexemes text[]) +RETURNS SETOF record AS +$$ +SELECT + tt.alias AS alias, + tt.description AS description, + parse.token AS token, + ARRAY ( SELECT m.mapdict::pg_catalog.regdictionary + FROM pg_catalog.pg_ts_config_map AS m + WHERE m.mapcfg = $1 AND m.maptokentype = parse.tokid + ORDER BY m.mapseqno ) + AS dictionaries, + ( SELECT mapdict::pg_catalog.regdictionary + FROM pg_catalog.pg_ts_config_map AS m + WHERE m.mapcfg = $1 AND m.maptokentype = parse.tokid + ORDER BY pg_catalog.ts_lexize(mapdict, parse.token) IS NULL, m.mapseqno + LIMIT 1 + ) AS dictionary, + ( SELECT pg_catalog.ts_lexize(mapdict, parse.token) + FROM pg_catalog.pg_ts_config_map AS m + WHERE m.mapcfg = $1 AND m.maptokentype = parse.tokid + ORDER BY pg_catalog.ts_lexize(mapdict, parse.token) IS NULL, m.mapseqno + LIMIT 1 + ) AS lexemes +FROM pg_catalog.ts_parse( + (SELECT cfgparser FROM pg_catalog.pg_ts_config WHERE oid = $1 ), $2 + ) AS parse, + pg_catalog.ts_token_type( + (SELECT cfgparser FROM pg_catalog.pg_ts_config WHERE oid = $1 ) + ) AS tt +WHERE tt.tokid = parse.tokid +$$ +LANGUAGE SQL STRICT STABLE PARALLEL SAFE; + +COMMENT ON FUNCTION ts_debug(regconfig,text) IS + 'debug function for text search configuration'; + +CREATE FUNCTION ts_debug(IN document text, + OUT alias text, + OUT description text, + OUT token text, + OUT dictionaries regdictionary[], + OUT dictionary regdictionary, + OUT lexemes text[]) +RETURNS SETOF record AS +$$ + SELECT * FROM pg_catalog.ts_debug( pg_catalog.get_current_ts_config(), $1); +$$ +LANGUAGE SQL STRICT STABLE PARALLEL SAFE; + +COMMENT ON FUNCTION ts_debug(text) IS + 'debug function for current text search configuration'; -- -- Redeclare built-in functions that need default values attached to their @@ -950,6 +1016,77 @@ GRANT SELECT (subdbid, subname, subowner, subenabled, subslotname, subpublicatio -- to get filled in.) -- +CREATE OR REPLACE FUNCTION + pg_start_backup(label text, fast boolean DEFAULT false, exclusive boolean DEFAULT true) + RETURNS pg_lsn STRICT VOLATILE LANGUAGE internal AS 'pg_start_backup' + PARALLEL RESTRICTED; + +CREATE OR REPLACE FUNCTION pg_stop_backup ( + exclusive boolean, wait_for_archive boolean DEFAULT true, + OUT lsn pg_lsn, OUT labelfile text, OUT spcmapfile text) + RETURNS SETOF record STRICT VOLATILE LANGUAGE internal as 'pg_stop_backup_v2' + PARALLEL RESTRICTED; + +-- legacy definition for compatibility with 9.3 +CREATE OR REPLACE FUNCTION + json_populate_record(base anyelement, from_json json, use_json_as_text boolean DEFAULT false) + RETURNS anyelement LANGUAGE internal STABLE AS 'json_populate_record' PARALLEL SAFE; + +-- legacy definition for compatibility with 9.3 +CREATE OR REPLACE FUNCTION + json_populate_recordset(base anyelement, from_json json, use_json_as_text boolean DEFAULT false) + RETURNS SETOF anyelement LANGUAGE internal STABLE ROWS 100 AS 'json_populate_recordset' PARALLEL SAFE; + +CREATE OR REPLACE FUNCTION pg_logical_slot_get_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data text) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_get_changes'; + +CREATE OR REPLACE FUNCTION pg_logical_slot_peek_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data text) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_peek_changes'; + +CREATE OR REPLACE FUNCTION pg_logical_slot_get_binary_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data bytea) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_get_binary_changes'; + +CREATE OR REPLACE FUNCTION pg_logical_slot_peek_binary_changes( + IN slot_name name, IN upto_lsn pg_lsn, IN upto_nchanges int, VARIADIC options text[] DEFAULT '{}', + OUT lsn pg_lsn, OUT xid xid, OUT data bytea) +RETURNS SETOF RECORD +LANGUAGE INTERNAL +VOLATILE ROWS 1000 COST 1000 +AS 'pg_logical_slot_peek_binary_changes'; + +CREATE OR REPLACE FUNCTION pg_create_physical_replication_slot( + IN slot_name name, IN immediately_reserve boolean DEFAULT false, + IN temporary boolean DEFAULT false, + OUT slot_name name, OUT lsn pg_lsn) +RETURNS RECORD +LANGUAGE INTERNAL +STRICT VOLATILE +AS 'pg_create_physical_replication_slot'; + +CREATE OR REPLACE FUNCTION pg_create_logical_replication_slot( + IN slot_name name, IN plugin name, + IN temporary boolean DEFAULT false, + OUT slot_name text, OUT lsn pg_lsn) +RETURNS RECORD +LANGUAGE INTERNAL +STRICT VOLATILE +AS 'pg_create_logical_replication_slot'; + CREATE OR REPLACE FUNCTION make_interval(years int4 DEFAULT 0, months int4 DEFAULT 0, weeks int4 DEFAULT 0, days int4 DEFAULT 0, hours int4 DEFAULT 0, mins int4 DEFAULT 0, @@ -982,6 +1119,58 @@ LANGUAGE INTERNAL STRICT IMMUTABLE PARALLEL SAFE AS 'jsonb_insert'; +-- +-- The default permissions for functions mean that anyone can execute them. +-- A number of functions shouldn't be executable by just anyone, but rather +-- than use explicit 'superuser()' checks in those functions, we use the GRANT +-- system to REVOKE access to those functions at initdb time. Administrators +-- can later change who can access these functions, or leave them as only +-- available to superuser / cluster owner, if they choose. +-- +REVOKE EXECUTE ON FUNCTION pg_start_backup(text, boolean, boolean) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stop_backup() FROM public; +REVOKE EXECUTE ON FUNCTION pg_stop_backup(boolean, boolean) FROM public; +REVOKE EXECUTE ON FUNCTION pg_create_restore_point(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_switch_wal() FROM public; +REVOKE EXECUTE ON FUNCTION pg_wal_replay_pause() FROM public; +REVOKE EXECUTE ON FUNCTION pg_wal_replay_resume() FROM public; +REVOKE EXECUTE ON FUNCTION pg_rotate_logfile() FROM public; +REVOKE EXECUTE ON FUNCTION pg_reload_conf() FROM public; +REVOKE EXECUTE ON FUNCTION pg_current_logfile() FROM public; +REVOKE EXECUTE ON FUNCTION pg_current_logfile(text) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_stat_reset() FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_reset_shared(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_reset_single_table_counters(oid) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_reset_single_function_counters(oid) FROM public; + +REVOKE EXECUTE ON FUNCTION lo_import(text) FROM public; +REVOKE EXECUTE ON FUNCTION lo_import(text, oid) FROM public; +REVOKE EXECUTE ON FUNCTION lo_export(oid, text) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_ls_logdir() FROM public; +REVOKE EXECUTE ON FUNCTION pg_ls_waldir() FROM public; + +REVOKE EXECUTE ON FUNCTION pg_read_file(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_file(text,bigint,bigint) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_file(text,bigint,bigint,boolean) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_read_binary_file(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_binary_file(text,bigint,bigint) FROM public; +REVOKE EXECUTE ON FUNCTION pg_read_binary_file(text,bigint,bigint,boolean) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_stat_file(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_stat_file(text,boolean) FROM public; + +REVOKE EXECUTE ON FUNCTION pg_ls_dir(text) FROM public; +REVOKE EXECUTE ON FUNCTION pg_ls_dir(text,boolean,boolean) FROM public; + +-- +-- We also set up some things as accessible to standard roles. +-- +GRANT EXECUTE ON FUNCTION pg_ls_logdir() TO pg_monitor; +GRANT EXECUTE ON FUNCTION pg_ls_waldir() TO pg_monitor; + GRANT pg_read_all_settings TO pg_monitor; GRANT pg_read_all_stats TO pg_monitor; GRANT pg_stat_scan_tables TO pg_monitor; diff --git a/src/postgres/src/backend/commands/copy.c b/src/postgres/src/backend/commands/copy.c index 3528585f5070..33060bf46c7a 100644 --- a/src/postgres/src/backend/commands/copy.c +++ b/src/postgres/src/backend/commands/copy.c @@ -857,8 +857,7 @@ DoCopy(ParseState *pstate, const CopyStmt *stmt, attnums = CopyGetAttnums(tupDesc, rel, stmt->attlist); foreach(cur, attnums) { - int attno = lfirst_int(cur) - - FirstLowInvalidHeapAttributeNumber; + int attno = lfirst_int(cur) - YBGetFirstLowInvalidAttributeNumber(rel); if (is_from) rte->insertedCols = bms_add_member(rte->insertedCols, attno); diff --git a/src/postgres/src/backend/commands/createas.c b/src/postgres/src/backend/commands/createas.c index f0cba94f87fc..644d6bdcb16d 100644 --- a/src/postgres/src/backend/commands/createas.c +++ b/src/postgres/src/backend/commands/createas.c @@ -540,7 +540,7 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo) for (attnum = 1; attnum <= intoRelationDesc->rd_att->natts; attnum++) rte->insertedCols = bms_add_member(rte->insertedCols, - attnum - FirstLowInvalidHeapAttributeNumber); + attnum - YBGetFirstLowInvalidAttributeNumber(intoRelationDesc)); ExecCheckRTPerms(list_make1(rte), true); diff --git a/src/postgres/src/backend/commands/sequence.c b/src/postgres/src/backend/commands/sequence.c index 95ec3c219abc..5e34eb5453d0 100644 --- a/src/postgres/src/backend/commands/sequence.c +++ b/src/postgres/src/backend/commands/sequence.c @@ -597,20 +597,11 @@ DeleteSequenceTuple(Oid relid) } HeapTuple -ReadSequenceTuple(Relation seqrel, bool check_permissions) +YBReadSequenceTuple(Relation seqrel) { /* Get sequence OID */ Oid relid = seqrel->rd_id; - /* Verify we can access it */ - if (check_permissions && - pg_class_aclcheck(relid, GetUserId(), - ACL_USAGE) != ACLCHECK_OK) - ereport(ERROR, - (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE), - errmsg("permission denied for sequence %s", - RelationGetRelationName(seqrel)))); - /* Read our data from YB's table of all sequences */ FormData_pg_sequence_data seqdataform; if (IsYugaByteEnabled()) diff --git a/src/postgres/src/backend/commands/trigger.c b/src/postgres/src/backend/commands/trigger.c index 209048870844..2b5d4ba208a8 100644 --- a/src/postgres/src/backend/commands/trigger.c +++ b/src/postgres/src/backend/commands/trigger.c @@ -3447,7 +3447,7 @@ TriggerEnabled(EState *estate, ResultRelInfo *relinfo, modified = false; for (i = 0; i < trigger->tgnattr; i++) { - if (bms_is_member(trigger->tgattr[i] - FirstLowInvalidHeapAttributeNumber, + if (bms_is_member(trigger->tgattr[i] - YBGetFirstLowInvalidAttributeNumber(relinfo->ri_RelationDesc), modifiedCols)) { modified = true; diff --git a/src/postgres/src/backend/commands/ybccmds.c b/src/postgres/src/backend/commands/ybccmds.c index 35f4dc041970..027b71a2a300 100644 --- a/src/postgres/src/backend/commands/ybccmds.c +++ b/src/postgres/src/backend/commands/ybccmds.c @@ -450,6 +450,7 @@ YBCPrepareAlterTable(AlterTableStmt *stmt, Relation rel, Oid relationId) case AT_DisableTrig: case AT_DisableTrigAll: case AT_DisableTrigUser: + case AT_ChangeOwner: /* For these cases a YugaByte alter isn't required, so we do nothing. */ break; diff --git a/src/postgres/src/backend/executor/execMain.c b/src/postgres/src/backend/executor/execMain.c index 6aa65cb88c4a..c528607e3492 100644 --- a/src/postgres/src/backend/executor/execMain.c +++ b/src/postgres/src/backend/executor/execMain.c @@ -95,7 +95,7 @@ static bool ExecCheckRTEPermsModified(Oid relOid, Oid userid, Bitmapset *modifiedCols, AclMode requiredPerms); static void ExecCheckXactReadOnly(PlannedStmt *plannedstmt); -static char *ExecBuildSlotValueDescription(Oid reloid, +static char *ExecBuildSlotValueDescription(Relation rel, TupleTableSlot *slot, TupleDesc tupdesc, Bitmapset *modifiedCols, @@ -676,8 +676,8 @@ ExecCheckRTEPerms(RangeTblEntry *rte) while ((col = bms_next_member(rte->selectedCols, col)) >= 0) { - /* bit #s are offset by FirstLowInvalidHeapAttributeNumber */ - AttrNumber attno = col + FirstLowInvalidHeapAttributeNumber; + /* Add appropriate offset to get attribute # from column # */ + AttrNumber attno = col + YBGetFirstLowInvalidAttributeNumberFromOid(relOid); if (attno == InvalidAttrNumber) { @@ -739,8 +739,8 @@ ExecCheckRTEPermsModified(Oid relOid, Oid userid, Bitmapset *modifiedCols, while ((col = bms_next_member(modifiedCols, col)) >= 0) { - /* bit #s are offset by FirstLowInvalidHeapAttributeNumber */ - AttrNumber attno = col + FirstLowInvalidHeapAttributeNumber; + /* Add appropriate offset to get attribute # from column # */ + AttrNumber attno = col + YBGetFirstLowInvalidAttributeNumberFromOid(relOid); if (attno == InvalidAttrNumber) { @@ -1955,7 +1955,7 @@ ExecPartitionCheckEmitError(ResultRelInfo *resultRelInfo, insertedCols = GetInsertedColumns(resultRelInfo, estate); updatedCols = GetUpdatedColumns(resultRelInfo, estate); modifiedCols = bms_union(insertedCols, updatedCols); - val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel), + val_desc = ExecBuildSlotValueDescription(rel, slot, tupdesc, modifiedCols, @@ -2034,7 +2034,7 @@ ExecConstraints(ResultRelInfo *resultRelInfo, insertedCols = GetInsertedColumns(resultRelInfo, estate); updatedCols = GetUpdatedColumns(resultRelInfo, estate); modifiedCols = bms_union(insertedCols, updatedCols); - val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel), + val_desc = ExecBuildSlotValueDescription(rel, slot, tupdesc, modifiedCols, @@ -2082,7 +2082,7 @@ ExecConstraints(ResultRelInfo *resultRelInfo, insertedCols = GetInsertedColumns(resultRelInfo, estate); updatedCols = GetUpdatedColumns(resultRelInfo, estate); modifiedCols = bms_union(insertedCols, updatedCols); - val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel), + val_desc = ExecBuildSlotValueDescription(rel, slot, tupdesc, modifiedCols, @@ -2188,7 +2188,7 @@ ExecWithCheckOptions(WCOKind kind, ResultRelInfo *resultRelInfo, insertedCols = GetInsertedColumns(resultRelInfo, estate); updatedCols = GetUpdatedColumns(resultRelInfo, estate); modifiedCols = bms_union(insertedCols, updatedCols); - val_desc = ExecBuildSlotValueDescription(RelationGetRelid(rel), + val_desc = ExecBuildSlotValueDescription(rel, slot, tupdesc, modifiedCols, @@ -2254,7 +2254,7 @@ ExecWithCheckOptions(WCOKind kind, ResultRelInfo *resultRelInfo, * columns they are. */ static char * -ExecBuildSlotValueDescription(Oid reloid, +ExecBuildSlotValueDescription(Relation rel, TupleTableSlot *slot, TupleDesc tupdesc, Bitmapset *modifiedCols, @@ -2268,6 +2268,7 @@ ExecBuildSlotValueDescription(Oid reloid, AclResult aclresult; bool table_perm = false; bool any_perm = false; + Oid reloid = RelationGetRelid(rel); /* * Check if RLS is enabled and should be active for the relation; if so, @@ -2322,7 +2323,7 @@ ExecBuildSlotValueDescription(Oid reloid, */ aclresult = pg_attribute_aclcheck(reloid, att->attnum, GetUserId(), ACL_SELECT); - if (bms_is_member(att->attnum - FirstLowInvalidHeapAttributeNumber, + if (bms_is_member(att->attnum - YBGetFirstLowInvalidAttributeNumber(rel), modifiedCols) || aclresult == ACLCHECK_OK) { column_perm = any_perm = true; diff --git a/src/postgres/src/backend/executor/ybcModifyTable.c b/src/postgres/src/backend/executor/ybcModifyTable.c index d154a75e7d9d..eaec003a0fa7 100644 --- a/src/postgres/src/backend/executor/ybcModifyTable.c +++ b/src/postgres/src/backend/executor/ybcModifyTable.c @@ -41,6 +41,7 @@ #include "catalog/pg_attribute.h" #include "catalog/pg_namespace.h" #include "catalog/pg_database.h" +#include "utils/catcache.h" #include "utils/inval.h" #include "utils/relcache.h" #include "utils/rel.h" @@ -51,6 +52,15 @@ #include "yb/yql/pggate/ybc_pggate.h" #include "pg_yb_utils.h" +/* + * Hack to ensure that the next CommandCounterIncrement() will call + * CommandEndInvalidationMessages(). The result of this call is not + * needed on the yb side, however the side effects are. + */ +void MarkCurrentCommandUsed() { + (void) GetCurrentCommandId(true); +} + /* * Returns whether a relation's attribute is a real column in the backing * YugaByte table. (It implies we can both read from and write to it). @@ -251,13 +261,23 @@ static bool IsSystemCatalogChange(Relation rel) static YBCStatus YBCExecWriteStmt(YBCPgStatement ybc_stmt, Relation rel) { bool is_syscatalog_change = IsSystemCatalogChange(rel); - bool is_syscatalog_version_change = false; + bool modifies_row = false; + HandleYBStmtStatus(YBCPgDmlModifiesRow(ybc_stmt, &modifies_row), ybc_stmt); + + /* + * If this write may invalidate catalog cache tuples (i.e. UPDATE or DELETE), + * or this write may insert into a cached list, we must increment the + * cache version so other sessions can invalidate their caches. + * NOTE: If this relation caches lists, an INSERT could effectively be + * UPDATINGing the list object. + */ + bool is_syscatalog_version_change = is_syscatalog_change + && (modifies_row || RelationHasCachedLists(rel)); /* Let the master know if this should increment the catalog version. */ - if (is_syscatalog_change) + if (is_syscatalog_version_change) { - YBCPgSetIfIsSysCatalogVersionChange(ybc_stmt, - &is_syscatalog_version_change); + HandleYBStmtStatus(YBCPgSetIsSysCatalogVersionChange(ybc_stmt), ybc_stmt); } HandleYBStmtStatus(YBCPgSetCatalogCacheVersion(ybc_stmt, @@ -279,6 +299,7 @@ static YBCStatus YBCExecWriteStmt(YBCPgStatement ybc_stmt, Relation rel) */ if (!status && is_syscatalog_version_change) { + // TODO(shane) also update the shared memory catalog version here. yb_catalog_cache_version += 1; } @@ -397,7 +418,7 @@ static Oid YBCExecuteInsertInternal(Relation rel, */ if (IsCatalogRelation(rel)) { - GetCurrentCommandId(true); + MarkCurrentCommandUsed(); CacheInvalidateHeapTuple(rel, tuple, NULL); } @@ -680,7 +701,7 @@ void YBCDeleteSysCatalogTuple(Relation rel, HeapTuple tuple) * boundary. Do this now so if there is an error with delete we will * re-query to get the correct state from the master. */ - GetCurrentCommandId(true); + MarkCurrentCommandUsed(); CacheInvalidateHeapTuple(rel, tuple, NULL); HandleYBStmtStatus(YBCExecWriteStmt(delete_stmt, rel), delete_stmt); @@ -734,7 +755,7 @@ void YBCUpdateSysCatalogTuple(Relation rel, HeapTuple oldtuple, HeapTuple tuple) * is an error with update we will re-query to get the correct state * from the master. */ - GetCurrentCommandId(true); + MarkCurrentCommandUsed(); if (oldtuple) CacheInvalidateHeapTuple(rel, oldtuple, tuple); else diff --git a/src/postgres/src/backend/libpq/pg_hba.conf.sample b/src/postgres/src/backend/libpq/pg_hba.conf.sample index c853e362329e..75f20070e5b5 100644 --- a/src/postgres/src/backend/libpq/pg_hba.conf.sample +++ b/src/postgres/src/backend/libpq/pg_hba.conf.sample @@ -76,6 +76,10 @@ # TYPE DATABASE USER ADDRESS METHOD +# YugaByte-specific rules to trust all incoming connections. +host all all 0.0.0.0/0 trust +host all all ::0/0 trust + @remove-line-for-nolocal@# "local" is for Unix domain socket connections only @remove-line-for-nolocal@local all all @authmethodlocal@ # IPv4 local connections: diff --git a/src/postgres/src/backend/optimizer/prep/prepunion.c b/src/postgres/src/backend/optimizer/prep/prepunion.c index c8465a14af59..55dd006ca830 100644 --- a/src/postgres/src/backend/optimizer/prep/prepunion.c +++ b/src/postgres/src/backend/optimizer/prep/prepunion.c @@ -2003,6 +2003,11 @@ translate_col_privs(const Bitmapset *parent_privs, int attno; ListCell *lc; + /* + * TODO check that these offsets (i.e. FirstLowInvalidHeapAttributeNumber) work + * properly for YugaByte tables after #1129 (specifically INHERITS). + */ + /* System attributes have the same numbers in all tables */ for (attno = FirstLowInvalidHeapAttributeNumber + 1; attno < 0; attno++) { diff --git a/src/postgres/src/backend/parser/analyze.c b/src/postgres/src/backend/parser/analyze.c index 6c14f4eccbd0..68d9c1e82750 100644 --- a/src/postgres/src/backend/parser/analyze.c +++ b/src/postgres/src/backend/parser/analyze.c @@ -867,7 +867,7 @@ transformInsertStmt(ParseState *pstate, InsertStmt *stmt) qry->targetList = lappend(qry->targetList, tle); rte->insertedCols = bms_add_member(rte->insertedCols, - attr_num - FirstLowInvalidHeapAttributeNumber); + attr_num - YBGetFirstLowInvalidAttributeNumber(pstate->p_target_relation)); icols = lnext(icols); attnos = lnext(attnos); @@ -2394,7 +2394,7 @@ transformUpdateTargetList(ParseState *pstate, List *origTlist) /* Mark the target column as requiring update permissions */ target_rte->updatedCols = bms_add_member(target_rte->updatedCols, - attrno - FirstLowInvalidHeapAttributeNumber); + attrno - YBGetFirstLowInvalidAttributeNumber(pstate->p_target_relation)); orig_tl = lnext(orig_tl); } diff --git a/src/postgres/src/backend/parser/gram.y b/src/postgres/src/backend/parser/gram.y index 4efc13dc0f37..af34ed2b0e21 100644 --- a/src/postgres/src/backend/parser/gram.y +++ b/src/postgres/src/backend/parser/gram.y @@ -896,6 +896,18 @@ stmt : | RemoveFuncStmt { parser_ybc_beta_feature(@1, "function"); } | CreateTrigStmt { parser_ybc_beta_feature(@1, "trigger"); } | CreateExtensionStmt { parser_ybc_beta_feature(@1, "extension"); } + | AlterDefaultPrivilegesStmt { parser_ybc_beta_feature(@1, "roles"); } + | AlterGroupStmt { parser_ybc_beta_feature(@1, "roles"); } + | AlterOwnerStmt { parser_ybc_beta_feature(@1, "roles"); } + | AlterRoleSetStmt { parser_ybc_beta_feature(@1, "roles"); } + | AlterRoleStmt { parser_ybc_beta_feature(@1, "roles"); } + | CreateGroupStmt { parser_ybc_beta_feature(@1, "roles"); } + | CreateRoleStmt { parser_ybc_beta_feature(@1, "roles"); } + | DropOwnedStmt { parser_ybc_beta_feature(@1, "roles"); } + | DropRoleStmt { parser_ybc_beta_feature(@1, "roles"); } + | GrantRoleStmt { parser_ybc_beta_feature(@1, "roles"); } + | ReassignOwnedStmt { parser_ybc_beta_feature(@1, "roles"); } + | RevokeRoleStmt { parser_ybc_beta_feature(@1, "roles"); } /* Not supported in template0/template1 statements */ | CreateAsStmt { parser_ybc_not_support_in_templates(@1, "This statement"); } @@ -905,7 +917,6 @@ stmt : /* Not supported statements */ | AlterEventTrigStmt { parser_ybc_signal_unsupported(@1, "This statement", 1156); } | AlterCollationStmt { parser_ybc_not_support(@1, "This statement"); } - | AlterDefaultPrivilegesStmt { parser_ybc_not_support(@1, "This statement"); } | AlterEnumStmt { parser_ybc_signal_unsupported(@1, "This statement", 1152); } | AlterExtensionStmt { parser_ybc_signal_unsupported(@1, "This statement", 1154); } | AlterExtensionContentsStmt { parser_ybc_signal_unsupported(@1, "This statement", 1154); } @@ -913,18 +924,14 @@ stmt : | AlterForeignServerStmt { parser_ybc_not_support(@1, "This statement"); } | AlterForeignTableStmt { parser_ybc_not_support(@1, "This statement"); } | AlterFunctionStmt { parser_ybc_signal_unsupported(@1, "This statement", 1155); } - | AlterGroupStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | AlterObjectDependsStmt { parser_ybc_not_support(@1, "This statement"); } | AlterObjectSchemaStmt { parser_ybc_not_support(@1, "This statement"); } - | AlterOwnerStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | AlterOperatorStmt { parser_ybc_not_support(@1, "This statement"); } | AlterPolicyStmt { parser_ybc_not_support(@1, "This statement"); } | AlterSystemStmt { parser_ybc_not_support(@1, "This statement"); } | AlterTblSpcStmt { parser_ybc_signal_unsupported(@1, "This statement", 1153); } | AlterCompositeTypeStmt { parser_ybc_not_support(@1, "This statement"); } | AlterPublicationStmt { parser_ybc_not_support(@1, "This statement"); } - | AlterRoleSetStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } - | AlterRoleStmt { parser_ybc_not_support(@1, "This statement"); } | AlterSubscriptionStmt { parser_ybc_not_support(@1, "This statement"); } | AlterTSConfigurationStmt { parser_ybc_not_support(@1, "This statement"); } | AlterTSDictionaryStmt { parser_ybc_not_support(@1, "This statement"); } @@ -940,7 +947,6 @@ stmt : | CreateFdwStmt { parser_ybc_not_support(@1, "This statement"); } | CreateForeignServerStmt { parser_ybc_not_support(@1, "This statement"); } | CreateForeignTableStmt { parser_ybc_not_support(@1, "This statement"); } - | CreateGroupStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | CreateMatViewStmt { parser_ybc_not_support(@1, "This statement"); } | CreateOpClassStmt { parser_ybc_not_support(@1, "This statement"); } | CreateOpFamilyStmt { parser_ybc_not_support(@1, "This statement"); } @@ -953,7 +959,6 @@ stmt : | CreateTableSpaceStmt { parser_ybc_signal_unsupported(@1, "This statement", 1153); } | CreateTransformStmt { parser_ybc_not_support(@1, "This statement"); } | CreateEventTrigStmt { parser_ybc_signal_unsupported(@1, "This statement", 1156); } - | CreateRoleStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | CreateUserMappingStmt { parser_ybc_not_support(@1, "This statement"); } | DeclareCursorStmt { parser_ybc_not_support(@1, "This statement"); } | DefineStmt { parser_ybc_not_support(@1, "This statement"); } @@ -961,25 +966,20 @@ stmt : | DropCastStmt { parser_ybc_not_support(@1, "This statement"); } | DropOpClassStmt { parser_ybc_not_support(@1, "This statement"); } | DropOpFamilyStmt { parser_ybc_not_support(@1, "This statement"); } - | DropOwnedStmt { parser_ybc_not_support(@1, "This statement"); } | DropPLangStmt { parser_ybc_not_support(@1, "This statement"); } | DropSubscriptionStmt { parser_ybc_not_support(@1, "This statement"); } | DropTableSpaceStmt { parser_ybc_signal_unsupported(@1, "This statement", 1153); } | DropTransformStmt { parser_ybc_not_support(@1, "This statement"); } - | DropRoleStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | DropUserMappingStmt { parser_ybc_not_support(@1, "This statement"); } | FetchStmt { parser_ybc_not_support(@1, "This statement"); } - | GrantRoleStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | ImportForeignSchemaStmt { parser_ybc_not_support(@1, "This statement"); } | ListenStmt { parser_ybc_not_support(@1, "This statement"); } | RefreshMatViewStmt { parser_ybc_not_support(@1, "This statement"); } | LoadStmt { parser_ybc_not_support(@1, "This statement"); } | NotifyStmt { parser_ybc_not_support(@1, "This statement"); } - | ReassignOwnedStmt { parser_ybc_not_support(@1, "This statement"); } | ReindexStmt { parser_ybc_not_support(@1, "This statement"); } | RemoveAggrStmt { parser_ybc_not_support(@1, "This statement"); } | RemoveOperStmt { parser_ybc_not_support(@1, "This statement"); } - | RevokeRoleStmt { parser_ybc_signal_unsupported(@1, "This statement", 869); } | RuleStmt { parser_ybc_not_support(@1, "This statement"); } | SecLabelStmt { parser_ybc_not_support(@1, "This statement"); } | UnlistenStmt { parser_ybc_not_support(@1, "This statement"); } @@ -1009,7 +1009,7 @@ CallStmt: CALL func_application CreateRoleStmt: CREATE ROLE RoleId opt_with OptRoleList { - parser_ybc_signal_unsupported(@1, "CREATE ROLE", 869); + parser_ybc_beta_feature(@1, "roles"); CreateRoleStmt *n = makeNode(CreateRoleStmt); n->stmt_type = ROLESTMT_ROLE; n->role = $3; @@ -1032,7 +1032,7 @@ opt_with: WITH {} OptRoleList: OptRoleList CreateOptRoleElem { - parser_ybc_signal_unsupported(@2, "CREATE USER ROLE with element", 869); + parser_ybc_beta_feature(@2, "roles"); $$ = lappend($1, $2); } | /* EMPTY */ { $$ = NIL; } @@ -1188,7 +1188,7 @@ CreateUserStmt: AlterRoleStmt: ALTER ROLE RoleSpec opt_with AlterOptRoleList { - parser_ybc_signal_unsupported(@1, "ALTER ROLE", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleStmt *n = makeNode(AlterRoleStmt); n->role = $3; n->action = +1; /* add, if there are members */ @@ -1197,7 +1197,7 @@ AlterRoleStmt: } | ALTER USER RoleSpec opt_with AlterOptRoleList { - parser_ybc_signal_unsupported(@1, "ALTER USER", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleStmt *n = makeNode(AlterRoleStmt); n->role = $3; n->action = +1; /* add, if there are members */ @@ -1214,7 +1214,7 @@ opt_in_database: AlterRoleSetStmt: ALTER ROLE RoleSpec opt_in_database SetResetClause { - parser_ybc_signal_unsupported(@1, "ALTER ROLE SET", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleSetStmt *n = makeNode(AlterRoleSetStmt); n->role = $3; n->database = $4; @@ -1223,7 +1223,7 @@ AlterRoleSetStmt: } | ALTER ROLE ALL opt_in_database SetResetClause { - parser_ybc_signal_unsupported(@1, "ALTER ROLE SET", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleSetStmt *n = makeNode(AlterRoleSetStmt); n->role = NULL; n->database = $4; @@ -1232,7 +1232,7 @@ AlterRoleSetStmt: } | ALTER USER RoleSpec opt_in_database SetResetClause { - parser_ybc_signal_unsupported(@1, "ALTER USER SET", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleSetStmt *n = makeNode(AlterRoleSetStmt); n->role = $3; n->database = $4; @@ -1241,7 +1241,7 @@ AlterRoleSetStmt: } | ALTER USER ALL opt_in_database SetResetClause { - parser_ybc_signal_unsupported(@1, "ALTER USER SET", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleSetStmt *n = makeNode(AlterRoleSetStmt); n->role = NULL; n->database = $4; @@ -1263,7 +1263,7 @@ AlterRoleSetStmt: DropRoleStmt: DROP ROLE role_list { - parser_ybc_signal_unsupported(@1, "DROP ROLE", 869); + parser_ybc_beta_feature(@1, "roles"); DropRoleStmt *n = makeNode(DropRoleStmt); n->missing_ok = false; n->roles = $3; @@ -1271,7 +1271,7 @@ DropRoleStmt: } | DROP ROLE IF_P EXISTS role_list { - parser_ybc_signal_unsupported(@1, "DROP ROLE", 869); + parser_ybc_beta_feature(@1, "roles"); DropRoleStmt *n = makeNode(DropRoleStmt); n->missing_ok = true; n->roles = $5; @@ -1279,7 +1279,7 @@ DropRoleStmt: } | DROP USER role_list { - parser_ybc_signal_unsupported(@1, "DROP USER", 869); + parser_ybc_beta_feature(@1, "roles"); DropRoleStmt *n = makeNode(DropRoleStmt); n->missing_ok = false; n->roles = $3; @@ -1287,7 +1287,7 @@ DropRoleStmt: } | DROP USER IF_P EXISTS role_list { - parser_ybc_signal_unsupported(@1, "DROP USER", 869); + parser_ybc_beta_feature(@1, "roles"); DropRoleStmt *n = makeNode(DropRoleStmt); n->roles = $5; n->missing_ok = true; @@ -1295,7 +1295,7 @@ DropRoleStmt: } | DROP GROUP_P role_list { - parser_ybc_signal_unsupported(@1, "DROP GROUP", 869); + parser_ybc_beta_feature(@1, "roles"); DropRoleStmt *n = makeNode(DropRoleStmt); n->missing_ok = false; n->roles = $3; @@ -1303,7 +1303,7 @@ DropRoleStmt: } | DROP GROUP_P IF_P EXISTS role_list { - parser_ybc_signal_unsupported(@1, "DROP GROUP", 869); + parser_ybc_beta_feature(@1, "roles"); DropRoleStmt *n = makeNode(DropRoleStmt); n->missing_ok = true; n->roles = $5; @@ -1321,7 +1321,7 @@ DropRoleStmt: CreateGroupStmt: CREATE GROUP_P RoleId opt_with OptRoleList { - parser_ybc_signal_unsupported(@1, "CREATE GROUP", 869); + parser_ybc_beta_feature(@1, "roles"); CreateRoleStmt *n = makeNode(CreateRoleStmt); n->stmt_type = ROLESTMT_GROUP; n->role = $3; @@ -1340,7 +1340,7 @@ CreateGroupStmt: AlterGroupStmt: ALTER GROUP_P RoleSpec add_drop USER role_list { - parser_ybc_signal_unsupported(@1, "ALTER GROUP", 869); + parser_ybc_beta_feature(@1, "roles"); AlterRoleStmt *n = makeNode(AlterRoleStmt); n->role = $3; n->action = $4; @@ -1364,7 +1364,7 @@ add_drop: ADD_P { $$ = +1; } CreateSchemaStmt: CREATE SCHEMA OptSchemaName AUTHORIZATION RoleSpec OptSchemaEltList { - parser_ybc_signal_unsupported(@1, "CREATE SCHEMA", 869); + parser_ybc_beta_feature(@1, "roles"); CreateSchemaStmt *n = makeNode(CreateSchemaStmt); /* One can omit the schema name or the authorization id. */ n->schemaname = $3; @@ -1385,7 +1385,7 @@ CreateSchemaStmt: } | CREATE SCHEMA IF_P NOT EXISTS OptSchemaName AUTHORIZATION RoleSpec OptSchemaEltList { - parser_ybc_signal_unsupported(@1, "CREATE SCHEMA with AUTHORIZATION", 869); + parser_ybc_beta_feature(@1, "roles"); CreateSchemaStmt *n = makeNode(CreateSchemaStmt); /* schema name can be omitted here, too */ n->schemaname = $6; @@ -1729,7 +1729,7 @@ reset_rest: } | SESSION AUTHORIZATION { - parser_ybc_not_support(@1, "RESET SESSION AUTHORIZATION"); + parser_ybc_beta_feature(@1, "roles"); VariableSetStmt *n = makeNode(VariableSetStmt); n->kind = VAR_RESET; n->name = "session_authorization"; @@ -2617,7 +2617,7 @@ alter_table_cmd: /* ALTER TABLE OWNER TO RoleSpec */ | OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER TABLE OWNER", 1124); + parser_ybc_beta_feature(@1, "roles"); AlterTableCmd *n = makeNode(AlterTableCmd); n->subtype = AT_ChangeOwner; n->newowner = $3; @@ -6524,7 +6524,7 @@ DropOpFamilyStmt: DropOwnedStmt: DROP OWNED BY role_list opt_drop_behavior { - parser_ybc_not_support(@1, "DROP OWNED BY"); + parser_ybc_beta_feature(@1, "roles"); DropOwnedStmt *n = makeNode(DropOwnedStmt); n->roles = $4; n->behavior = $5; @@ -6535,7 +6535,7 @@ DropOwnedStmt: ReassignOwnedStmt: REASSIGN OWNED BY role_list TO RoleSpec { - parser_ybc_not_support(@1, "REASSIGN OWNED BY"); + parser_ybc_beta_feature(@1, "roles"); ReassignOwnedStmt *n = makeNode(ReassignOwnedStmt); n->roles = $4; n->newrole = $6; @@ -7647,7 +7647,7 @@ opt_grant_grant_option: GrantRoleStmt: GRANT privilege_list TO role_list opt_grant_admin_option opt_granted_by { - parser_ybc_signal_unsupported(@1, "GRANT", 869); + parser_ybc_beta_feature(@1, "roles"); GrantRoleStmt *n = makeNode(GrantRoleStmt); n->is_grant = true; n->granted_roles = $2; @@ -7661,7 +7661,7 @@ GrantRoleStmt: RevokeRoleStmt: REVOKE privilege_list FROM role_list opt_granted_by opt_drop_behavior { - parser_ybc_signal_unsupported(@1, "REVOKE", 869); + parser_ybc_beta_feature(@1, "roles"); GrantRoleStmt *n = makeNode(GrantRoleStmt); n->is_grant = false; n->admin_opt = false; @@ -7672,7 +7672,7 @@ RevokeRoleStmt: } | REVOKE ADMIN OPTION FOR privilege_list FROM role_list opt_granted_by opt_drop_behavior { - parser_ybc_signal_unsupported(@1, "REVOKE", 869); + parser_ybc_beta_feature(@1, "roles"); GrantRoleStmt *n = makeNode(GrantRoleStmt); n->is_grant = false; n->admin_opt = true; @@ -7700,7 +7700,7 @@ opt_granted_by: GRANTED BY RoleSpec { $$ = $3; } AlterDefaultPrivilegesStmt: ALTER DEFAULT PRIVILEGES DefACLOptionList DefACLAction { - parser_ybc_not_support(@1, "ALTER DEFAULT PRIVILEGES"); + parser_ybc_beta_feature(@1, "roles"); AlterDefaultPrivilegesStmt *n = makeNode(AlterDefaultPrivilegesStmt); n->options = $4; n->action = (GrantStmt *) $5; @@ -9057,7 +9057,7 @@ RenameStmt: ALTER AGGREGATE aggregate_with_argtypes RENAME TO name } | ALTER GROUP_P RoleId RENAME TO RoleId { - parser_ybc_not_support(@1, "ALTER GROUP"); + parser_ybc_beta_feature(@1, "roles"); RenameStmt *n = makeNode(RenameStmt); n->renameType = OBJECT_ROLE; n->subname = $3; @@ -9434,7 +9434,7 @@ RenameStmt: ALTER AGGREGATE aggregate_with_argtypes RENAME TO name } | ALTER ROLE RoleId RENAME TO RoleId { - parser_ybc_not_support(@1, "ALTER ROLE"); + parser_ybc_beta_feature(@1, "roles"); RenameStmt *n = makeNode(RenameStmt); n->renameType = OBJECT_ROLE; n->subname = $3; @@ -9444,7 +9444,7 @@ RenameStmt: ALTER AGGREGATE aggregate_with_argtypes RENAME TO name } | ALTER USER RoleId RENAME TO RoleId { - parser_ybc_not_support(@1, "ALTER USER"); + parser_ybc_beta_feature(@1, "roles"); RenameStmt *n = makeNode(RenameStmt); n->renameType = OBJECT_ROLE; n->subname = $3; @@ -9932,7 +9932,7 @@ operator_def_arg: AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER AGGREGATE OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_AGGREGATE; n->object = (Node *) $3; @@ -9941,7 +9941,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER COLLATION any_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER COLLATION OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_COLLATION; n->object = (Node *) $3; @@ -9950,7 +9950,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER CONVERSION_P any_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER CONVERSION OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_CONVERSION; n->object = (Node *) $3; @@ -9959,7 +9959,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER DATABASE database_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER DATABASE OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_DATABASE; n->object = (Node *) makeString($3); @@ -9968,7 +9968,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER DOMAIN_P any_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER DOMAIN OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_DOMAIN; n->object = (Node *) $3; @@ -9977,7 +9977,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER FUNCTION function_with_argtypes OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER FUNCTION OWNER", 1155); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_FUNCTION; n->object = (Node *) $3; @@ -9986,7 +9986,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER opt_procedural LANGUAGE name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER LANGUAGE OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_LANGUAGE; n->object = (Node *) makeString($4); @@ -9995,7 +9995,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER LARGE_P OBJECT_P NumericOnly OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER LARGE OBJECT OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_LARGEOBJECT; n->object = (Node *) $4; @@ -10004,7 +10004,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER OPERATOR operator_with_argtypes OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER OPERATOR OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_OPERATOR; n->object = (Node *) $3; @@ -10013,7 +10013,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER OPERATOR CLASS any_name USING access_method OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER OPERATOR CLASS OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_OPCLASS; n->object = (Node *) lcons(makeString($6), $4); @@ -10022,7 +10022,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER OPERATOR FAMILY any_name USING access_method OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER OPERATOR FAMILY OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_OPFAMILY; n->object = (Node *) lcons(makeString($6), $4); @@ -10031,7 +10031,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER PROCEDURE function_with_argtypes OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER PROCEDURE OWNER", 1155); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_PROCEDURE; n->object = (Node *) $3; @@ -10040,7 +10040,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER ROUTINE function_with_argtypes OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER ROUTINE OWNER", 1155); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_ROUTINE; n->object = (Node *) $3; @@ -10049,7 +10049,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER SCHEMA name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER SCHEMA OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_SCHEMA; n->object = (Node *) makeString($3); @@ -10058,7 +10058,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER TYPE_P any_name OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER TYPE OWNER", 1152); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_TYPE; n->object = (Node *) $3; @@ -10067,7 +10067,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER TABLESPACE name OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER TABLESPACE OWNER", 1153); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_TABLESPACE; n->object = (Node *) makeString($3); @@ -10076,7 +10076,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER STATISTICS any_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER STATISTICS OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_STATISTIC_EXT; n->object = (Node *) $3; @@ -10085,7 +10085,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER TEXT_P SEARCH DICTIONARY any_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER TEXT SEARCH DICTIONARY OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_TSDICTIONARY; n->object = (Node *) $5; @@ -10094,7 +10094,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER TEXT_P SEARCH CONFIGURATION any_name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER TEXT SEARCH CONFIGURATION OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_TSCONFIGURATION; n->object = (Node *) $5; @@ -10103,7 +10103,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER FOREIGN DATA_P WRAPPER name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER TEXT SEARCH WRAPPER OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_FDW; n->object = (Node *) makeString($5); @@ -10112,7 +10112,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER SERVER name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER SERVER OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_FOREIGN_SERVER; n->object = (Node *) makeString($3); @@ -10121,7 +10121,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER EVENT TRIGGER name OWNER TO RoleSpec { - parser_ybc_signal_unsupported(@1, "ALTER EVENT TRIGGER OWNER", 1156); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_EVENT_TRIGGER; n->object = (Node *) makeString($4); @@ -10130,7 +10130,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER PUBLICATION name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER PUBLICATION OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_PUBLICATION; n->object = (Node *) makeString($3); @@ -10139,7 +10139,7 @@ AlterOwnerStmt: ALTER AGGREGATE aggregate_with_argtypes OWNER TO RoleSpec } | ALTER SUBSCRIPTION name OWNER TO RoleSpec { - parser_ybc_not_support(@1, "ALTER SUBSCRIPTION OWNER"); + parser_ybc_beta_feature(@1, "roles"); AlterOwnerStmt *n = makeNode(AlterOwnerStmt); n->objectType = OBJECT_SUBSCRIPTION; n->object = (Node *) makeString($3); diff --git a/src/postgres/src/backend/parser/parse_clause.c b/src/postgres/src/backend/parser/parse_clause.c index c951fc2769c8..bd85f5fefec9 100644 --- a/src/postgres/src/backend/parser/parse_clause.c +++ b/src/postgres/src/backend/parser/parse_clause.c @@ -3230,11 +3230,10 @@ transformOnConflictArbiter(ParseState *pstate, */ if (infer->conname) { - Oid relid = RelationGetRelid(pstate->p_target_relation); RangeTblEntry *rte = pstate->p_target_rangetblentry; Bitmapset *conattnos; - conattnos = get_relation_constraint_attnos(relid, infer->conname, + conattnos = get_relation_constraint_attnos(pstate->p_target_relation, infer->conname, false, constraint); /* Make sure the rel as a whole is marked for SELECT access */ diff --git a/src/postgres/src/backend/storage/buffer/bufmgr.c b/src/postgres/src/backend/storage/buffer/bufmgr.c index 8f591c36e1dc..35c24cfbe88f 100644 --- a/src/postgres/src/backend/storage/buffer/bufmgr.c +++ b/src/postgres/src/backend/storage/buffer/bufmgr.c @@ -598,7 +598,7 @@ ReadBuffer(Relation reln, BlockNumber blockNum) } /* Only here for sequence support */ -extern HeapTuple ReadSequenceTuple(Relation seqrel, bool check_permissions); +extern HeapTuple YBReadSequenceTuple(Relation seqrel); /* * ReadBufferExtended -- returns a buffer containing the requested @@ -663,13 +663,13 @@ ReadBufferExtended(Relation reln, ForkNumber forkNum, BlockNumber blockNum, /* Special handling for sequences */ if (RelationGetForm(reln)->relkind == RELKIND_SEQUENCE) { + /* Get a sequence tuple */ + HeapTuple seqtuple = YBReadSequenceTuple(reln); + /* Create an empty buffer to initialize with the sequence data */ buf = ReadBuffer_common(reln->rd_smgr, reln->rd_rel->relpersistence, forkNum, blockNum, RBM_ZERO_AND_LOCK, strategy, &hit); - /* Get a sequence tuple */ - HeapTuple seqtuple = ReadSequenceTuple(reln, true); - /* Insert onto the page */ Page dp = BufferGetPage(buf); PageInit(dp, BLCKSZ, sizeof(*seqtuple)); diff --git a/src/postgres/src/backend/tcop/postgres.c b/src/postgres/src/backend/tcop/postgres.c index b2910d13c1d2..2bc407f3c321 100644 --- a/src/postgres/src/backend/tcop/postgres.c +++ b/src/postgres/src/backend/tcop/postgres.c @@ -71,6 +71,7 @@ #include "tcop/pquery.h" #include "tcop/tcopprot.h" #include "tcop/utility.h" +#include "utils/inval.h" #include "utils/relcache.h" #include "utils/catcache.h" #include "utils/syscache.h" @@ -3696,8 +3697,9 @@ static void YBRefreshCache() /* Need to execute some (read) queries internally so start a local txn. */ start_xact_command(); - /* Clear and reload system catalog caches. */ + /* Clear and reload system catalog caches, including all callbacks. */ ResetCatalogCaches(); + CallSystemCacheCallbacks(); YBPreloadRelCache(); /* Also invalidate the pggate cache. */ diff --git a/src/postgres/src/backend/utils/adt/ri_triggers.c b/src/postgres/src/backend/utils/adt/ri_triggers.c index f80493a5ce6e..ab19e585c227 100644 --- a/src/postgres/src/backend/utils/adt/ri_triggers.c +++ b/src/postgres/src/backend/utils/adt/ri_triggers.c @@ -1939,10 +1939,10 @@ RI_Initial_Check(Trigger *trigger, Relation fk_rel, Relation pk_rel) { int attno; - attno = riinfo->pk_attnums[i] - FirstLowInvalidHeapAttributeNumber; + attno = riinfo->pk_attnums[i] - YBGetFirstLowInvalidAttributeNumber(pk_rel); pkrte->selectedCols = bms_add_member(pkrte->selectedCols, attno); - attno = riinfo->fk_attnums[i] - FirstLowInvalidHeapAttributeNumber; + attno = riinfo->fk_attnums[i] - YBGetFirstLowInvalidAttributeNumber(fk_rel); fkrte->selectedCols = bms_add_member(fkrte->selectedCols, attno); } diff --git a/src/postgres/src/backend/utils/cache/catcache.c b/src/postgres/src/backend/utils/cache/catcache.c index eedf4b76706e..d3a96ab20b6b 100644 --- a/src/postgres/src/backend/utils/cache/catcache.c +++ b/src/postgres/src/backend/utils/cache/catcache.c @@ -2444,6 +2444,33 @@ PrepareToInvalidateCacheTuple(Relation relation, } } +/* + * RelationHasCachedLists + * + * Returns true if there is a catalog cache associated with this + * relation which is currently caching at least one list. + */ +bool +RelationHasCachedLists(Relation relation) +{ + slist_iter iter; + Oid reloid; + + /* sanity checks */ + Assert(RelationIsValid(relation)); + Assert(CacheHdr != NULL); + + reloid = RelationGetRelid(relation); + + slist_foreach(iter, &CacheHdr->ch_caches) + { + CatCache *ccp = slist_container(CatCache, cc_next, iter.cur); + if (ccp->cc_reloid == reloid && !dlist_is_empty(&ccp->cc_lists)) + return true; + } + + return false; +} /* * Subroutines for warning about reference leaks. These are exported so diff --git a/src/postgres/src/backend/utils/cache/inval.c b/src/postgres/src/backend/utils/cache/inval.c index 80bc7740fe8c..568b15c5b5b7 100644 --- a/src/postgres/src/backend/utils/cache/inval.c +++ b/src/postgres/src/backend/utils/cache/inval.c @@ -629,25 +629,17 @@ LocalExecuteInvalidationMessage(SharedInvalidationMessage *msg) } /* - * InvalidateSystemCaches - * - * This blows away all tuples in the system catalog caches and - * all the cached relation descriptors and smgr cache entries. - * Relation descriptors that have positive refcounts are then rebuilt. + * CallSystemCacheCallbacks * - * We call this when we see a shared-inval-queue overflow signal, - * since that tells us we've lost some shared-inval messages and hence - * don't know what needs to be invalidated. + * Calls all syscache and relcache invalidation callbacks. + * This is useful when the entire cache is being reloaded or + * invalidated, rather than a single cache entry. */ void -InvalidateSystemCaches(void) +CallSystemCacheCallbacks(void) { int i; - InvalidateCatalogSnapshot(); - ResetCatalogCaches(); - RelationCacheInvalidate(); /* gets smgr and relmap too */ - for (i = 0; i < syscache_callback_count; i++) { struct SYSCACHECALLBACK *ccitem = syscache_callback_list + i; @@ -663,6 +655,26 @@ InvalidateSystemCaches(void) } } +/* + * InvalidateSystemCaches + * + * This blows away all tuples in the system catalog caches and + * all the cached relation descriptors and smgr cache entries. + * Relation descriptors that have positive refcounts are then rebuilt. + * + * We call this when we see a shared-inval-queue overflow signal, + * since that tells us we've lost some shared-inval messages and hence + * don't know what needs to be invalidated. + */ +void +InvalidateSystemCaches(void) +{ + InvalidateCatalogSnapshot(); + ResetCatalogCaches(); + RelationCacheInvalidate(); /* gets smgr and relmap too */ + CallSystemCacheCallbacks(); +} + /* ---------------------------------------------------------------- * public functions diff --git a/src/postgres/src/bin/initdb/initdb.c b/src/postgres/src/bin/initdb/initdb.c index f2c7107901ff..c182aeb9ad51 100644 --- a/src/postgres/src/bin/initdb/initdb.c +++ b/src/postgres/src/bin/initdb/initdb.c @@ -3165,10 +3165,10 @@ initialize_data_directory(void) setup_conversion(cmdfd); setup_dictionary(cmdfd); - - setup_privileges(cmdfd); } + setup_privileges(cmdfd); + setup_schema(cmdfd); load_plpgsql(cmdfd); diff --git a/src/postgres/src/include/access/ybcam.h b/src/postgres/src/include/access/ybcam.h index 5b40f6e7bf75..d5f9943edb96 100644 --- a/src/postgres/src/include/access/ybcam.h +++ b/src/postgres/src/include/access/ybcam.h @@ -55,6 +55,9 @@ typedef struct YbScanDescData Relation index; + /* Oid of the table being scanned */ + Oid tableOid; + /* Kept execution control to pass it to PgGate. * - When YBC-index-scan layer is called by Postgres IndexScan functions, it will read the * "yb_exec_params" from Postgres IndexScan and kept the info in this attribute. diff --git a/src/postgres/src/include/catalog/pg_constraint.h b/src/postgres/src/include/catalog/pg_constraint.h index cdea401bafbc..30b996ed8c70 100644 --- a/src/postgres/src/include/catalog/pg_constraint.h +++ b/src/postgres/src/include/catalog/pg_constraint.h @@ -245,7 +245,7 @@ extern void AlterConstraintNamespaces(Oid ownerId, Oid oldNspId, extern void ConstraintSetParentConstraint(Oid childConstrId, Oid parentConstrId); extern Oid get_relation_constraint_oid(Oid relid, const char *conname, bool missing_ok); -extern Bitmapset *get_relation_constraint_attnos(Oid relid, const char *conname, +extern Bitmapset *get_relation_constraint_attnos(Relation rel, const char *conname, bool missing_ok, Oid *constraintOid); extern Oid get_domain_constraint_oid(Oid typid, const char *conname, bool missing_ok); extern Oid get_relation_idx_constraint_oid(Oid relationId, Oid indexId); diff --git a/src/postgres/src/include/utils/catcache.h b/src/postgres/src/include/utils/catcache.h index 79b1259949a6..4da3898ee9c2 100644 --- a/src/postgres/src/include/utils/catcache.h +++ b/src/postgres/src/include/utils/catcache.h @@ -229,6 +229,7 @@ extern void PrepareToInvalidateCacheTuple(Relation relation, HeapTuple tuple, HeapTuple newtuple, void (*function) (int, uint32, Oid)); +extern bool RelationHasCachedLists(Relation relation); extern void PrintCatCacheLeakWarning(HeapTuple tuple); extern void PrintCatCacheListLeakWarning(CatCList *list); diff --git a/src/postgres/src/include/utils/inval.h b/src/postgres/src/include/utils/inval.h index 7a66d466f703..94bf1d7b26b5 100644 --- a/src/postgres/src/include/utils/inval.h +++ b/src/postgres/src/include/utils/inval.h @@ -62,5 +62,7 @@ extern void CacheRegisterRelcacheCallback(RelcacheCallbackFunction func, extern void CallSyscacheCallbacks(int cacheid, uint32 hashvalue); +extern void CallSystemCacheCallbacks(void); + extern void InvalidateSystemCaches(void); #endif /* INVAL_H */ diff --git a/src/postgres/src/test/regress/expected/yb_init_privs.out b/src/postgres/src/test/regress/expected/yb_init_privs.out new file mode 100644 index 000000000000..292b1a1035b1 --- /dev/null +++ b/src/postgres/src/test/regress/expected/yb_init_privs.out @@ -0,0 +1,12 @@ +-- Test initial privileges +-- There should always be some initial privileges, set up by initdb +SELECT count(*) > 0 FROM pg_init_privs; + ?column? +---------- + t +(1 row) + +-- Intentionally include some non-initial privs for pg_dump to dump out +GRANT SELECT ON pg_proc TO CURRENT_USER; +GRANT SELECT (prosrc) ON pg_proc TO CURRENT_USER; +GRANT SELECT (rolname, rolsuper) ON pg_authid TO CURRENT_USER; diff --git a/src/postgres/src/test/regress/expected/yb_password.out b/src/postgres/src/test/regress/expected/yb_password.out new file mode 100644 index 000000000000..393d836eada1 --- /dev/null +++ b/src/postgres/src/test/regress/expected/yb_password.out @@ -0,0 +1,105 @@ +-- +-- Tests for password verifiers +-- +-- Tests for GUC password_encryption +SET password_encryption = 'novalue'; -- error +ERROR: invalid value for parameter "password_encryption": "novalue" +HINT: Available values: md5, scram-sha-256. +SET password_encryption = true; -- ok +SET password_encryption = 'md5'; -- ok +SET password_encryption = 'scram-sha-256'; -- ok +-- consistency of password entries +SET password_encryption = 'md5'; +CREATE ROLE regress_passwd1 PASSWORD 'role_pwd1'; +SET password_encryption = 'on'; +CREATE ROLE regress_passwd2 PASSWORD 'role_pwd2'; +SET password_encryption = 'scram-sha-256'; +CREATE ROLE regress_passwd3 PASSWORD 'role_pwd3'; +CREATE ROLE regress_passwd4 PASSWORD NULL; +-- check list of created entries +-- +-- The scram verifier will look something like: +-- SCRAM-SHA-256$4096:E4HxLGtnRzsYwg==$6YtlR4t69SguDiwFvbVgVZtuz6gpJQQqUMZ7IQJK5yI=:ps75jrHeYU4lXCcXI4O8oIdJ3eO8o2jirjruw9phBTo= +-- +-- Since the salt is random, the exact value stored will be different on every test +-- run. Use a regular expression to mask the changing parts. +SELECT rolname, regexp_replace(rolpassword, '(SCRAM-SHA-256)\$(\d+):([a-zA-Z0-9+/=]+)\$([a-zA-Z0-9+=/]+):([a-zA-Z0-9+/=]+)', '\1$\2:$:') as rolpassword_masked + FROM pg_authid + WHERE rolname LIKE 'regress_passwd%' + ORDER BY rolname, rolpassword; + rolname | rolpassword_masked +-----------------+--------------------------------------------------- + regress_passwd1 | md5783277baca28003b33453252be4dbb34 + regress_passwd2 | md54044304ba511dd062133eb5b4b84a2a3 + regress_passwd3 | SCRAM-SHA-256$4096:$: + regress_passwd4 | +(4 rows) + +-- Rename a role +ALTER ROLE regress_passwd2 RENAME TO regress_passwd2_new; +NOTICE: MD5 password cleared because of role rename +-- md5 entry should have been removed +SELECT rolname, rolpassword + FROM pg_authid + WHERE rolname LIKE 'regress_passwd2_new' + ORDER BY rolname, rolpassword; + rolname | rolpassword +---------------------+------------- + regress_passwd2_new | +(1 row) + +ALTER ROLE regress_passwd2_new RENAME TO regress_passwd2; +-- Change passwords with ALTER USER. With plaintext or already-encrypted +-- passwords. +SET password_encryption = 'md5'; +-- encrypt with MD5 +ALTER ROLE regress_passwd2 PASSWORD 'foo'; +-- already encrypted, use as they are +ALTER ROLE regress_passwd1 PASSWORD 'md5cd3578025fe2c3d7ed1b9a9b26238b70'; +ALTER ROLE regress_passwd3 PASSWORD 'SCRAM-SHA-256$4096:VLK4RMaQLCvNtQ==$6YtlR4t69SguDiwFvbVgVZtuz6gpJQQqUMZ7IQJK5yI=:ps75jrHeYU4lXCcXI4O8oIdJ3eO8o2jirjruw9phBTo='; +SET password_encryption = 'scram-sha-256'; +-- create SCRAM verifier +ALTER ROLE regress_passwd4 PASSWORD 'foo'; +-- already encrypted with MD5, use as it is +CREATE ROLE regress_passwd5 PASSWORD 'md5e73a4b11df52a6068f8b39f90be36023'; +SELECT rolname, regexp_replace(rolpassword, '(SCRAM-SHA-256)\$(\d+):([a-zA-Z0-9+/=]+)\$([a-zA-Z0-9+=/]+):([a-zA-Z0-9+/=]+)', '\1$\2:$:') as rolpassword_masked + FROM pg_authid + WHERE rolname LIKE 'regress_passwd%' + ORDER BY rolname, rolpassword; + rolname | rolpassword_masked +-----------------+--------------------------------------------------- + regress_passwd1 | md5cd3578025fe2c3d7ed1b9a9b26238b70 + regress_passwd2 | md5dfa155cadd5f4ad57860162f3fab9cdb + regress_passwd3 | SCRAM-SHA-256$4096:$: + regress_passwd4 | SCRAM-SHA-256$4096:$: + regress_passwd5 | md5e73a4b11df52a6068f8b39f90be36023 +(5 rows) + +-- An empty password is not allowed, in any form +CREATE ROLE regress_passwd_empty PASSWORD ''; +NOTICE: empty string is not a valid password, clearing password +ALTER ROLE regress_passwd_empty PASSWORD 'md585939a5ce845f1a1b620742e3c659e0a'; +NOTICE: empty string is not a valid password, clearing password +ALTER ROLE regress_passwd_empty PASSWORD 'SCRAM-SHA-256$4096:hpFyHTUsSWcR7O9P$LgZFIt6Oqdo27ZFKbZ2nV+vtnYM995pDh9ca6WSi120=:qVV5NeluNfUPkwm7Vqat25RjSPLkGeoZBQs6wVv+um4='; +NOTICE: empty string is not a valid password, clearing password +SELECT rolpassword FROM pg_authid WHERE rolname='regress_passwd_empty'; + rolpassword +------------- + +(1 row) + +DROP ROLE regress_passwd1; +DROP ROLE regress_passwd2; +DROP ROLE regress_passwd3; +DROP ROLE regress_passwd4; +DROP ROLE regress_passwd5; +DROP ROLE regress_passwd_empty; +-- all entries should have been removed +SELECT rolname, rolpassword + FROM pg_authid + WHERE rolname LIKE 'regress_passwd%' + ORDER BY rolname, rolpassword; + rolname | rolpassword +---------+------------- +(0 rows) + diff --git a/src/postgres/src/test/regress/expected/yb_privileges.out b/src/postgres/src/test/regress/expected/yb_privileges.out new file mode 100644 index 000000000000..ff9faa6bb877 --- /dev/null +++ b/src/postgres/src/test/regress/expected/yb_privileges.out @@ -0,0 +1,1980 @@ +-- +-- Test access privileges +-- +-- Clean up in case a prior regression run failed +-- Suppress NOTICE messages when users/groups don't exist +SET client_min_messages TO 'warning'; +DROP ROLE IF EXISTS regress_priv_group1; +DROP ROLE IF EXISTS regress_priv_group2; +DROP ROLE IF EXISTS regress_priv_user1; +DROP ROLE IF EXISTS regress_priv_user2; +DROP ROLE IF EXISTS regress_priv_user3; +DROP ROLE IF EXISTS regress_priv_user4; +DROP ROLE IF EXISTS regress_priv_user5; +DROP ROLE IF EXISTS regress_priv_user6; +SELECT lo_unlink(oid) FROM pg_largeobject_metadata WHERE oid >= 1000 AND oid < 3000 ORDER BY oid; + lo_unlink +----------- +(0 rows) + +RESET client_min_messages; +-- test proper begins here +CREATE USER regress_priv_user1; +CREATE USER regress_priv_user2; +CREATE USER regress_priv_user3; +CREATE USER regress_priv_user4; +CREATE USER regress_priv_user5; +CREATE USER regress_priv_user5; -- duplicate +ERROR: role "regress_priv_user5" already exists +CREATE GROUP regress_priv_group1; +CREATE GROUP regress_priv_group2 WITH USER regress_priv_user1, regress_priv_user2; +ALTER GROUP regress_priv_group1 ADD USER regress_priv_user4; +ALTER GROUP regress_priv_group2 ADD USER regress_priv_user2; -- duplicate +NOTICE: role "regress_priv_user2" is already a member of role "regress_priv_group2" +ALTER GROUP regress_priv_group2 DROP USER regress_priv_user2; +GRANT regress_priv_group2 TO regress_priv_user4 WITH ADMIN OPTION; +-- test owner privileges +SET SESSION AUTHORIZATION regress_priv_user1; +SELECT session_user, current_user; + session_user | current_user +--------------------+-------------------- + regress_priv_user1 | regress_priv_user1 +(1 row) + +CREATE TABLE atest1 ( a int, b text ); +SELECT * FROM atest1; + a | b +---+--- +(0 rows) + +INSERT INTO atest1 VALUES (1, 'one'); +DELETE FROM atest1; +UPDATE atest1 SET a = 1 WHERE b = 'blech'; +TRUNCATE atest1; +REVOKE ALL ON atest1 FROM PUBLIC; +SELECT * FROM atest1; + a | b +---+--- +(0 rows) + +GRANT ALL ON atest1 TO regress_priv_user2; +GRANT SELECT ON atest1 TO regress_priv_user3, regress_priv_user4; +SELECT * FROM atest1; + a | b +---+--- +(0 rows) + +CREATE TABLE atest2 (col1 varchar(10), col2 boolean); +GRANT SELECT ON atest2 TO regress_priv_user2; +GRANT UPDATE ON atest2 TO regress_priv_user3; +GRANT INSERT ON atest2 TO regress_priv_user4; +GRANT TRUNCATE ON atest2 TO regress_priv_user5; +SET SESSION AUTHORIZATION regress_priv_user2; +SELECT session_user, current_user; + session_user | current_user +--------------------+-------------------- + regress_priv_user2 | regress_priv_user2 +(1 row) + +-- try various combinations of queries on atest1 and atest2 +SELECT * FROM atest1; -- ok + a | b +---+--- +(0 rows) + +SELECT * FROM atest2; -- ok + col1 | col2 +------+------ +(0 rows) + +INSERT INTO atest1 VALUES (2, 'two'); -- ok +INSERT INTO atest2 VALUES ('foo', true); -- fail +ERROR: permission denied for table atest2 +INSERT INTO atest1 SELECT 1, b FROM atest1; -- ok +UPDATE atest1 SET a = 1 WHERE a = 2; -- ok +UPDATE atest2 SET col2 = NOT col2; -- fail +ERROR: permission denied for table atest2 +DELETE FROM atest2; -- fail +ERROR: permission denied for table atest2 +TRUNCATE atest2; -- fail +ERROR: permission denied for table atest2 +COPY atest2 FROM stdin; -- fail +ERROR: permission denied for table atest2 +GRANT ALL ON atest1 TO PUBLIC; -- fail +WARNING: no privileges were granted for "atest1" +-- checks in subquery, both ok +SELECT * FROM atest1 WHERE ( b IN ( SELECT col1 FROM atest2 ) ); + a | b +---+--- +(0 rows) + +SELECT * FROM atest2 WHERE ( col1 IN ( SELECT b FROM atest1 ) ); + col1 | col2 +------+------ +(0 rows) + +SET SESSION AUTHORIZATION regress_priv_user3; +SELECT session_user, current_user; + session_user | current_user +--------------------+-------------------- + regress_priv_user3 | regress_priv_user3 +(1 row) + +SELECT * FROM atest1; -- ok + a | b +---+----- + 1 | two + 1 | two +(2 rows) + +SELECT * FROM atest2; -- fail +ERROR: permission denied for table atest2 +INSERT INTO atest1 VALUES (2, 'two'); -- fail +ERROR: permission denied for table atest1 +INSERT INTO atest2 VALUES ('foo', true); -- fail +ERROR: permission denied for table atest2 +INSERT INTO atest1 SELECT 1, b FROM atest1; -- fail +ERROR: permission denied for table atest1 +UPDATE atest1 SET a = 1 WHERE a = 2; -- fail +ERROR: permission denied for table atest1 +UPDATE atest2 SET col2 = NULL; -- ok +UPDATE atest2 SET col2 = NOT col2; -- fails; requires SELECT on atest2 +ERROR: permission denied for table atest2 +DELETE FROM atest2; -- fail +ERROR: permission denied for table atest2 +TRUNCATE atest2; -- fail +ERROR: permission denied for table atest2 +COPY atest2 FROM stdin; -- fail +ERROR: permission denied for table atest2 +-- checks in subquery, both fail +SELECT * FROM atest1 WHERE ( b IN ( SELECT col1 FROM atest2 ) ); +ERROR: permission denied for table atest2 +SELECT * FROM atest2 WHERE ( col1 IN ( SELECT b FROM atest1 ) ); +ERROR: permission denied for table atest2 +SET SESSION AUTHORIZATION regress_priv_user4; +COPY atest2 FROM stdin; -- ok +SELECT * FROM atest1; -- ok + a | b +---+----- + 1 | two + 1 | two +(2 rows) + +-- NOT SUPPORTED +-- +-- -- test leaky-function protections in selfuncs +-- +-- -- regress_priv_user1 will own a table and provide a view for it. +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- +-- CREATE TABLE atest12 as +-- SELECT x AS a, 10001 - x AS b FROM generate_series(1,10000) x; +-- CREATE INDEX ON atest12 (a); +-- CREATE INDEX ON atest12 (abs(a)); +-- +-- CREATE FUNCTION leak(integer,integer) RETURNS boolean +-- AS $$begin return $1 < $2; end$$ +-- LANGUAGE plpgsql immutable; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE OPERATOR <<< (procedure = leak, leftarg = integer, rightarg = integer, + restrict = scalarltsel); +ERROR: CREATE OPERATOR not supported yet +LINE 1: CREATE OPERATOR <<< (procedure = leak, leftarg = integer, ri... + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- +-- -- view with leaky operator +-- CREATE VIEW atest12v AS +-- SELECT * FROM atest12 WHERE b <<< 5; +-- GRANT SELECT ON atest12v TO PUBLIC; +-- +-- -- This plan should use nestloop, knowing that few rows will be selected. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12v x, atest12v y WHERE x.a = y.b; +-- +-- -- And this one. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12 x, atest12 y +-- WHERE x.a = y.b and abs(y.a) <<< 5; +-- +-- -- Check if regress_priv_user2 can break security. +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- +-- CREATE FUNCTION leak2(integer,integer) RETURNS boolean +-- AS $$begin raise notice 'leak % %', $1, $2; return $1 > $2; end$$ +-- LANGUAGE plpgsql immutable; +-- CREATE OPERATOR >>> (procedure = leak2, leftarg = integer, rightarg = integer, +-- restrict = scalargtsel); +-- +-- -- This should not show any "leak" notices before failing. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12 WHERE a >>> 0; +-- +-- -- This plan should use hashjoin, as it will expect many rows to be selected. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12v x, atest12v y WHERE x.a = y.b; +-- +-- -- Now regress_priv_user1 grants sufficient access to regress_priv_user2. +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- GRANT SELECT (a, b) ON atest12 TO PUBLIC; +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- +-- -- Now regress_priv_user2 will also get a good row estimate. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12v x, atest12v y WHERE x.a = y.b; +-- +-- -- But not for this, due to lack of table-wide permissions needed +-- -- to make use of the expression index's statistics. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12 x, atest12 y +-- WHERE x.a = y.b and abs(y.a) <<< 5; +-- +-- clean up (regress_priv_user1's objects are all dropped later) +-- DROP FUNCTION leak2(integer, integer) CASCADE; +-- +-- groups +SET SESSION AUTHORIZATION regress_priv_user3; +CREATE TABLE atest3 (one int, two int, three int); +GRANT DELETE ON atest3 TO GROUP regress_priv_group2; +SET SESSION AUTHORIZATION regress_priv_user1; +SELECT * FROM atest3; -- fail +ERROR: permission denied for table atest3 +DELETE FROM atest3; -- ok +-- views +SET SESSION AUTHORIZATION regress_priv_user3; +CREATE VIEW atestv1 AS SELECT * FROM atest1; -- ok +/* The next *should* fail, but it's not implemented that way yet. */ +CREATE VIEW atestv2 AS SELECT * FROM atest2; +CREATE VIEW atestv3 AS SELECT * FROM atest3; -- ok +/* Empty view is a corner case that failed in 9.2. */ +CREATE VIEW atestv0 AS SELECT 0 as x WHERE false; -- ok +SELECT * FROM atestv1; -- ok + a | b +---+----- + 1 | two + 1 | two +(2 rows) + +SELECT * FROM atestv2; -- fail +ERROR: permission denied for table atest2 +GRANT SELECT ON atestv1, atestv3 TO regress_priv_user4; +GRANT SELECT ON atestv2 TO regress_priv_user2; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT * FROM atestv1; -- ok + a | b +---+----- + 1 | two + 1 | two +(2 rows) + +SELECT * FROM atestv2; -- fail +ERROR: permission denied for view atestv2 +SELECT * FROM atestv3; -- ok + one | two | three +-----+-----+------- +(0 rows) + +SELECT * FROM atestv0; -- fail +ERROR: permission denied for view atestv0 +RESET SESSION AUTHORIZATION; +CREATE TABLE priv_int8_tbl(q1 int8, q2 int8, PRIMARY KEY (q1, q2)); +INSERT INTO priv_int8_tbl VALUES(' 123 ',' 456'); +INSERT INTO priv_int8_tbl VALUES('123 ','4567890123456789'); +INSERT INTO priv_int8_tbl VALUES('4567890123456789','123'); +INSERT INTO priv_int8_tbl VALUES(+4567890123456789,'4567890123456789'); +INSERT INTO priv_int8_tbl VALUES('+4567890123456789','-4567890123456789'); +SET SESSION AUTHORIZATION regress_priv_user4; +-- Appendrels excluded by constraints failed to check permissions in 8.4-9.2. +select * from + ((select a.q1 as x from priv_int8_tbl a offset 0) + union all + (select b.q2 as x from priv_int8_tbl b offset 0)) ss +where false; +ERROR: permission denied for table priv_int8_tbl +set constraint_exclusion = on; +select * from + ((select a.q1 as x, random() from priv_int8_tbl a where q1 > 0) + union all + (select b.q2 as x, random() from priv_int8_tbl b where q2 > 0)) ss +where x < 0; +ERROR: permission denied for table priv_int8_tbl +reset constraint_exclusion; +CREATE VIEW atestv4 AS SELECT * FROM atestv3; -- nested view +SELECT * FROM atestv4; -- ok + one | two | three +-----+-----+------- +(0 rows) + +GRANT SELECT ON atestv4 TO regress_priv_user2; +SET SESSION AUTHORIZATION regress_priv_user2; +-- Two complex cases: +SELECT * FROM atestv3; -- fail +ERROR: permission denied for view atestv3 +SELECT * FROM atestv4; -- ok (even though regress_priv_user2 cannot access underlying atestv3) + one | two | three +-----+-----+------- +(0 rows) + +SELECT * FROM atest2; -- ok + col1 | col2 +------+------ + bar | t +(1 row) + +SELECT * FROM atestv2; -- fail (even though regress_priv_user2 can access underlying atest2) +ERROR: permission denied for table atest2 +-- Test column level permissions +SET SESSION AUTHORIZATION regress_priv_user1; +CREATE SEQUENCE twoseq START 1000; +CREATE SEQUENCE fourseq START 100; +CREATE TABLE atest5 ( + one int, + two int UNIQUE NOT NULL DEFAULT NEXTVAL('twoseq'), + three int, + four int UNIQUE NOT NULL DEFAULT NEXTVAL('fourseq') +); +CREATE TABLE atest6 (one int, two int, blue int); +GRANT SELECT (one), INSERT (two), UPDATE (three) ON atest5 TO regress_priv_user4; +GRANT ALL (one) ON atest5 TO regress_priv_user3; +GRANT ALL ON SEQUENCE twoseq TO regress_priv_user3, regress_priv_user4; +GRANT ALL ON SEQUENCE fourseq TO regress_priv_user3, regress_priv_user4; +INSERT INTO atest5 VALUES (1,2,3); +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT * FROM atest5; -- fail +ERROR: permission denied for table atest5 +SELECT one FROM atest5; -- ok + one +----- + 1 +(1 row) + +COPY atest5 (one) TO stdout; -- ok +1 +SELECT two FROM atest5; -- fail +ERROR: permission denied for table atest5 +COPY atest5 (two) TO stdout; -- fail +ERROR: permission denied for table atest5 +SELECT atest5 FROM atest5; -- fail +ERROR: permission denied for table atest5 +COPY atest5 (one,two) TO stdout; -- fail +ERROR: permission denied for table atest5 +SELECT 1 FROM atest5; -- ok + ?column? +---------- + 1 +(1 row) + +SELECT 1 FROM atest5 a JOIN atest5 b USING (one); -- ok + ?column? +---------- + 1 +(1 row) + +SELECT 1 FROM atest5 a JOIN atest5 b USING (two); -- fail +ERROR: permission denied for table atest5 +SELECT 1 FROM atest5 a NATURAL JOIN atest5 b; -- fail +ERROR: permission denied for table atest5 +SELECT (j.*) IS NULL FROM (atest5 a JOIN atest5 b USING (one)) j; -- fail +ERROR: permission denied for table atest5 +SELECT 1 FROM atest5 WHERE two = 2; -- fail +ERROR: permission denied for table atest5 +SELECT * FROM atest1, atest5; -- fail +ERROR: permission denied for table atest5 +SELECT atest1.* FROM atest1, atest5; -- ok + a | b +---+----- + 1 | two + 1 | two +(2 rows) + +SELECT atest1.*,atest5.one FROM atest1, atest5; -- ok + a | b | one +---+-----+----- + 1 | two | 1 + 1 | two | 1 +(2 rows) + +SELECT atest1.*,atest5.one FROM atest1 JOIN atest5 ON (atest1.a = atest5.two); -- fail +ERROR: permission denied for table atest5 +SELECT atest1.*,atest5.one FROM atest1 JOIN atest5 ON (atest1.a = atest5.one); -- ok + a | b | one +---+-----+----- + 1 | two | 1 + 1 | two | 1 +(2 rows) + +SELECT one, two FROM atest5; -- fail +ERROR: permission denied for table atest5 +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT SELECT (one,two) ON atest6 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT one, two FROM atest5 NATURAL JOIN atest6; -- fail still +ERROR: permission denied for table atest5 +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT SELECT (two) ON atest5 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT one, two FROM atest5 NATURAL JOIN atest6; -- ok now + one | two +-----+----- +(0 rows) + +-- test column-level privileges for INSERT and UPDATE +-- TODO enable after #1611 +-- UPDATE atest5 SET three = 10; -- ok +INSERT INTO atest5 (two) VALUES (3); -- ok +COPY atest5 FROM stdin; -- fail +ERROR: permission denied for table atest5 +COPY atest5 (two) FROM stdin; -- ok +INSERT INTO atest5 (three) VALUES (4); -- fail +ERROR: permission denied for table atest5 +INSERT INTO atest5 VALUES (5,5,5); -- fail +ERROR: permission denied for table atest5 +UPDATE atest5 SET one = 8; -- fail +ERROR: permission denied for table atest5 +UPDATE atest5 SET three = 5, one = 2; -- fail +ERROR: permission denied for table atest5 +-- Check that column level privs are enforced in RETURNING +-- Ok. +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = 10; +-- Error. No SELECT on column three. +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = 10 RETURNING atest5.three; +ERROR: permission denied for table atest5 +-- Ok. May SELECT on column "one": +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = 10 RETURNING atest5.one; + one +----- + +(1 row) + +-- Check that column level privileges are enforced for EXCLUDED +-- Ok. we may select one +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = EXCLUDED.one; +-- Error. No select rights on three +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = EXCLUDED.three; +ERROR: permission denied for table atest5 +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set one = 8; -- fails (due to UPDATE) +ERROR: permission denied for table atest5 +INSERT INTO atest5(three) VALUES (4) ON CONFLICT (two) DO UPDATE set three = 10; -- fails (due to INSERT) +ERROR: permission denied for table atest5 +-- Check that the columns in the inference require select privileges +INSERT INTO atest5(four) VALUES (4); -- fail +ERROR: permission denied for table atest5 +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT INSERT (four) ON atest5 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +INSERT INTO atest5(four) VALUES (4) ON CONFLICT (four) DO UPDATE set three = 3; -- fails (due to SELECT) +ERROR: permission denied for table atest5 +INSERT INTO atest5(four) VALUES (4) ON CONFLICT ON CONSTRAINT atest5_four_key DO UPDATE set three = 3; -- fails (due to SELECT) +ERROR: permission denied for table atest5 +INSERT INTO atest5(four) VALUES (4); -- ok +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT SELECT (four) ON atest5 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +INSERT INTO atest5(four) VALUES (4) ON CONFLICT (four) DO UPDATE set three = 3; -- ok +INSERT INTO atest5(four) VALUES (4) ON CONFLICT ON CONSTRAINT atest5_four_key DO UPDATE set three = 3; -- ok +SET SESSION AUTHORIZATION regress_priv_user1; +REVOKE ALL (one) ON atest5 FROM regress_priv_user4; +GRANT SELECT (one,two,blue) ON atest6 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT one FROM atest5; -- fail +ERROR: permission denied for table atest5 +UPDATE atest5 SET one = 1; -- fail +ERROR: permission denied for table atest5 +SELECT atest6 FROM atest6; -- ok + atest6 +-------- +(0 rows) + +COPY atest6 TO stdout; -- ok +-- check error reporting with column privs +SET SESSION AUTHORIZATION regress_priv_user1; +CREATE TABLE t1 (c1 int, c2 int, c3 int check (c3 < 5), primary key (c1, c2)); +GRANT SELECT (c1) ON t1 TO regress_priv_user2; +GRANT INSERT (c1, c2, c3) ON t1 TO regress_priv_user2; +GRANT UPDATE (c1, c2, c3) ON t1 TO regress_priv_user2; +-- seed data +INSERT INTO t1 VALUES (1, 1, 1); +INSERT INTO t1 VALUES (1, 2, 1); +INSERT INTO t1 VALUES (2, 1, 2); +INSERT INTO t1 VALUES (2, 2, 2); +INSERT INTO t1 VALUES (3, 1, 3); +SET SESSION AUTHORIZATION regress_priv_user2; +INSERT INTO t1 (c1, c2) VALUES (1, 1); -- fail, but row not shown +ERROR: duplicate key value violates unique constraint "t1_pkey" +INSERT INTO t1 (c1, c2) VALUES (null, null); -- fail, but see columns being inserted +ERROR: null value in column "c1" violates not-null constraint +DETAIL: Failing row contains (c1, c2) = (null, null). +INSERT INTO t1 (c3) VALUES (null); -- fail, but see columns being inserted or have SELECT +ERROR: null value in column "c1" violates not-null constraint +DETAIL: Failing row contains (c1, c3) = (null, null). +INSERT INTO t1 (c1) VALUES (5); -- fail, but see columns being inserted or have SELECT +ERROR: null value in column "c2" violates not-null constraint +DETAIL: Failing row contains (c1) = (5). +UPDATE t1 SET c3 = 10; -- fail, but see columns with SELECT rights, or being modified +SET SESSION AUTHORIZATION regress_priv_user1; +DROP TABLE t1; +-- test column-level privileges when involved with DELETE +SET SESSION AUTHORIZATION regress_priv_user1; +ALTER TABLE atest6 ADD COLUMN three integer; +GRANT DELETE ON atest5 TO regress_priv_user3; +GRANT SELECT (two) ON atest5 TO regress_priv_user3; +REVOKE ALL (one) ON atest5 FROM regress_priv_user3; +GRANT SELECT (one) ON atest5 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT atest6 FROM atest6; -- fail +ERROR: permission denied for table atest6 +SELECT one FROM atest5 NATURAL JOIN atest6; -- fail +ERROR: permission denied for table atest5 +SET SESSION AUTHORIZATION regress_priv_user1; +ALTER TABLE atest6 DROP COLUMN three; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT atest6 FROM atest6; -- ok + atest6 +-------- +(0 rows) + +SELECT one FROM atest5 NATURAL JOIN atest6; -- ok + one +----- +(0 rows) + +SET SESSION AUTHORIZATION regress_priv_user1; +ALTER TABLE atest6 DROP COLUMN two; +REVOKE SELECT (one,blue) ON atest6 FROM regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT * FROM atest6; -- fail +ERROR: permission denied for table atest6 +SELECT 1 FROM atest6; -- fail +ERROR: permission denied for table atest6 +SET SESSION AUTHORIZATION regress_priv_user3; +DELETE FROM atest5 WHERE one = 1; -- fail +ERROR: permission denied for table atest5 +DELETE FROM atest5 WHERE two = 2; -- ok +-- NOT SUPPORTED +-- +-- -- check inheritance cases +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE TABLE atestp1 (f1 int, f2 int) WITH OIDS; +ERROR: OIDs are not supported for user tables. +-- CREATE TABLE atestp2 (fx int, fy int) WITH OIDS; +-- CREATE TABLE atestc (fz int) INHERITS (atestp1, atestp2); +-- GRANT SELECT(fx,fy,oid) ON atestp2 TO regress_priv_user2; +-- GRANT SELECT(fx) ON atestc TO regress_priv_user2; +-- +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- SELECT fx FROM atestp2; -- ok +-- SELECT fy FROM atestp2; -- ok +-- SELECT atestp2 FROM atestp2; -- ok +-- SELECT oid FROM atestp2; -- ok +-- SELECT fy FROM atestc; -- fail +-- +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- GRANT SELECT(fy,oid) ON atestc TO regress_priv_user2; +-- +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- SELECT fx FROM atestp2; -- still ok +-- SELECT fy FROM atestp2; -- ok +-- SELECT atestp2 FROM atestp2; -- ok +-- SELECT oid FROM atestp2; -- ok +-- +-- privileges on functions, languages +-- switch to superuser +\c - +REVOKE ALL PRIVILEGES ON LANGUAGE sql FROM PUBLIC; +GRANT USAGE ON LANGUAGE sql TO regress_priv_user1; -- ok +GRANT USAGE ON LANGUAGE c TO PUBLIC; -- fail +ERROR: language "c" is not trusted +DETAIL: GRANT and REVOKE are not allowed on untrusted languages, because only superusers can use untrusted languages. +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT USAGE ON LANGUAGE sql TO regress_priv_user2; -- fail +WARNING: no privileges were granted for "sql" +CREATE FUNCTION priv_testfunc1(int) RETURNS int AS 'select 2 * $1;' LANGUAGE sql; +CREATE FUNCTION priv_testfunc2(int) RETURNS int AS 'select 3 * $1;' LANGUAGE sql; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE AGGREGATE priv_testagg1(int) (sfunc = int4pl, stype = int4); +ERROR: CREATE AGGREGATE not supported yet +LINE 1: CREATE AGGREGATE priv_testagg1(int) (sfunc = int4pl, stype =... + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- +CREATE PROCEDURE priv_testproc1(int) AS 'select $1;' LANGUAGE sql; +REVOKE ALL ON FUNCTION priv_testfunc1(int), priv_testfunc2(int) FROM PUBLIC; +GRANT EXECUTE ON FUNCTION priv_testfunc1(int), priv_testfunc2(int) TO regress_priv_user2; +REVOKE ALL ON FUNCTION priv_testproc1(int) FROM PUBLIC; -- fail, not a function +ERROR: priv_testproc1(integer) is not a function +REVOKE ALL ON PROCEDURE priv_testproc1(int) FROM PUBLIC; +GRANT EXECUTE ON PROCEDURE priv_testproc1(int) TO regress_priv_user2; +GRANT USAGE ON FUNCTION priv_testfunc1(int) TO regress_priv_user3; -- semantic error +ERROR: invalid privilege type USAGE for function +-- GRANT USAGE ON FUNCTION priv_testagg1(int) TO regress_priv_user3; -- semantic error +GRANT USAGE ON PROCEDURE priv_testproc1(int) TO regress_priv_user3; -- semantic error +ERROR: invalid privilege type USAGE for procedure +GRANT ALL PRIVILEGES ON FUNCTION priv_testfunc1(int) TO regress_priv_user4; +GRANT ALL PRIVILEGES ON FUNCTION priv_testfunc_nosuch(int) TO regress_priv_user4; +ERROR: function priv_testfunc_nosuch(integer) does not exist +-- GRANT ALL PRIVILEGES ON FUNCTION priv_testagg1(int) TO regress_priv_user4; +GRANT ALL PRIVILEGES ON PROCEDURE priv_testproc1(int) TO regress_priv_user4; +CREATE FUNCTION priv_testfunc4(boolean) RETURNS text + AS 'select col1 from atest2 where col2 = $1;' + LANGUAGE sql SECURITY DEFINER; +GRANT EXECUTE ON FUNCTION priv_testfunc4(boolean) TO regress_priv_user3; +SET SESSION AUTHORIZATION regress_priv_user2; +SELECT priv_testfunc1(5), priv_testfunc2(5); -- ok + priv_testfunc1 | priv_testfunc2 +----------------+---------------- + 10 | 15 +(1 row) + +CREATE FUNCTION priv_testfunc3(int) RETURNS int AS 'select 2 * $1;' LANGUAGE sql; -- fail +ERROR: permission denied for language sql +-- SELECT priv_testagg1(x) FROM (VALUES (1), (2), (3)) _(x); -- ok +CALL priv_testproc1(6); -- ok +SET SESSION AUTHORIZATION regress_priv_user3; +SELECT priv_testfunc1(5); -- fail +ERROR: permission denied for function priv_testfunc1 +-- SELECT priv_testagg1(x) FROM (VALUES (1), (2), (3)) _(x); -- fail +CALL priv_testproc1(6); -- fail +ERROR: permission denied for procedure priv_testproc1 +SELECT col1 FROM atest2 WHERE col2 = true; -- fail +ERROR: permission denied for table atest2 +SELECT priv_testfunc4(true); -- ok + priv_testfunc4 +---------------- + bar +(1 row) + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT priv_testfunc1(5); -- ok + priv_testfunc1 +---------------- + 10 +(1 row) + +-- SELECT priv_testagg1(x) FROM (VALUES (1), (2), (3)) _(x); -- ok +CALL priv_testproc1(6); -- ok +DROP FUNCTION priv_testfunc1(int); -- fail +ERROR: must be owner of function priv_testfunc1 +-- DROP AGGREGATE priv_testagg1(int); -- fail +DROP PROCEDURE priv_testproc1(int); -- fail +ERROR: must be owner of procedure priv_testproc1 +\c - +DROP FUNCTION priv_testfunc1(int); -- ok +-- restore to sanity +GRANT ALL PRIVILEGES ON LANGUAGE sql TO PUBLIC; +-- verify privilege checks on array-element coercions +SELECT '{1}'::int4[]::int8[]; + int8 +------ + {1} +(1 row) + +REVOKE ALL ON FUNCTION int8(integer) FROM PUBLIC; +SELECT '{1}'::int4[]::int8[]; --superuser, suceed + int8 +------ + {1} +(1 row) + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT '{1}'::int4[]::int8[]; --other user, fail +ERROR: permission denied for function int8 +RESET SESSION AUTHORIZATION; +GRANT ALL ON FUNCTION int8(integer) TO PUBLIC; +SET SESSION AUTHORIZATION regress_priv_user4; +-- NOT SUPPORTED +-- +-- -- privileges on types +-- +-- -- switch to superuser +-- \c - +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE TYPE priv_testtype1 AS (a int, b text); +ERROR: CREATE TYPE not supported yet +LINE 1: CREATE TYPE priv_testtype1 AS (a int, b text); + ^ +HINT: See https://github.com/YugaByte/yugabyte-db/issues/1152. Click '+' on the description to raise its priority +-- REVOKE USAGE ON TYPE priv_testtype1 FROM PUBLIC; +-- GRANT USAGE ON TYPE priv_testtype1 TO regress_priv_user2; +-- GRANT USAGE ON TYPE _priv_testtype1 TO regress_priv_user2; -- fail +-- GRANT USAGE ON DOMAIN priv_testtype1 TO regress_priv_user2; -- fail +-- +-- CREATE DOMAIN priv_testdomain1 AS int; +-- REVOKE USAGE on DOMAIN priv_testdomain1 FROM PUBLIC; +-- GRANT USAGE ON DOMAIN priv_testdomain1 TO regress_priv_user2; +-- GRANT USAGE ON TYPE priv_testdomain1 TO regress_priv_user2; -- ok +-- +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- +-- -- commands that should fail +-- +-- CREATE DOMAIN priv_testdomain2a AS priv_testdomain1; +-- +-- +-- CREATE DOMAIN priv_testdomain3a AS int; +-- CREATE FUNCTION castfunc(int) RETURNS priv_testdomain3a AS $$ SELECT $1::priv_testdomain3a $$ LANGUAGE SQL; +-- CREATE CAST (priv_testdomain1 AS priv_testdomain3a) WITH FUNCTION castfunc(int); +-- DROP FUNCTION castfunc(int) CASCADE; +-- DROP DOMAIN priv_testdomain3a; +-- +-- CREATE FUNCTION priv_testfunc5a(a priv_testdomain1) RETURNS int LANGUAGE SQL AS $$ SELECT $1 $$; +-- CREATE FUNCTION priv_testfunc6a(b int) RETURNS priv_testdomain1 LANGUAGE SQL AS $$ SELECT $1::priv_testdomain1 $$; +-- +-- CREATE OPERATOR !+! (PROCEDURE = int4pl, LEFTARG = priv_testdomain1, RIGHTARG = priv_testdomain1); +-- +-- CREATE TABLE test5a (a int, b priv_testdomain1); +-- CREATE TABLE test6a OF priv_testtype1; +-- CREATE TABLE test10a (a int[], b priv_testtype1[]); +-- +-- CREATE TABLE test9a (a int, b int); +-- ALTER TABLE test9a ADD COLUMN c priv_testdomain1; +-- ALTER TABLE test9a ALTER COLUMN b TYPE priv_testdomain1; +-- +-- CREATE TYPE test7a AS (a int, b priv_testdomain1); +-- +-- CREATE TYPE test8a AS (a int, b int); +-- ALTER TYPE test8a ADD ATTRIBUTE c priv_testdomain1; +-- ALTER TYPE test8a ALTER ATTRIBUTE b TYPE priv_testdomain1; +-- +-- CREATE TABLE test11a AS (SELECT 1::priv_testdomain1 AS a); +-- +-- REVOKE ALL ON TYPE priv_testtype1 FROM PUBLIC; +-- +CREATE DOMAIN priv_testdomain1 AS int; +SET SESSION AUTHORIZATION regress_priv_user2; +-- commands that should succeed +CREATE DOMAIN priv_testdomain2b AS priv_testdomain1; +CREATE DOMAIN priv_testdomain3b AS int; +-- NOT SUPPORTED +-- +-- CREATE FUNCTION castfunc(int) RETURNS priv_testdomain3b AS $$ SELECT $1::priv_testdomain3b $$ LANGUAGE SQL; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE CAST (priv_testdomain1 AS priv_testdomain3b) WITH FUNCTION castfunc(int); +ERROR: CREATE CAST not supported yet +LINE 1: CREATE CAST (priv_testdomain1 AS priv_testdomain3b) WITH FUN... + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- +-- CREATE FUNCTION priv_testfunc5b(a priv_testdomain1) RETURNS int LANGUAGE SQL AS $$ SELECT $1 $$; +-- CREATE FUNCTION priv_testfunc6b(b int) RETURNS priv_testdomain1 LANGUAGE SQL AS $$ SELECT $1::priv_testdomain1 $$; +-- +-- CREATE OPERATOR !! (PROCEDURE = priv_testfunc5b, RIGHTARG = priv_testdomain1); +-- +CREATE TABLE test5b (a int, b priv_testdomain1); +CREATE TABLE test9b (a int, b int); +ALTER TABLE test9b ADD COLUMN c priv_testdomain1; +-- NOT SUPPORTED +-- +-- ALTER TABLE test9b ALTER COLUMN b TYPE priv_testdomain1; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE TYPE test7b AS (a int, b priv_testdomain1); +ERROR: CREATE TYPE not supported yet +LINE 1: CREATE TYPE test7b AS (a int, b priv_testdomain1); + ^ +HINT: See https://github.com/YugaByte/yugabyte-db/issues/1152. Click '+' on the description to raise its priority +-- +-- CREATE TYPE test8b AS (a int, b int); +-- ALTER TYPE test8b ADD ATTRIBUTE c priv_testdomain1; +-- ALTER TYPE test8b ALTER ATTRIBUTE b TYPE priv_testdomain1; +-- +CREATE TABLE test11b AS (SELECT 1::priv_testdomain1 AS a); +\c - +DROP DOMAIN priv_testdomain2b; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +DROP OPERATOR !! (NONE, priv_testdomain1); +ERROR: DROP OPERATOR not supported yet +LINE 1: DROP OPERATOR !! (NONE, priv_testdomain1); + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- DROP FUNCTION priv_testfunc5b(a priv_testdomain1); +-- DROP FUNCTION priv_testfunc6b(b int); +-- +DROP TABLE test5b; +DROP TABLE test9b; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +DROP TYPE test7b; +ERROR: DROP TYPE not supported yet +LINE 1: DROP TYPE test7b; + ^ +HINT: See https://github.com/YugaByte/yugabyte-db/issues/1152. Click '+' on the description to raise its priority +-- DROP TYPE test8b; +-- DROP CAST (priv_testdomain1 AS priv_testdomain3b); +-- DROP FUNCTION castfunc(int) CASCADE; +-- +DROP DOMAIN priv_testdomain3b; +DROP TABLE test11b; +DROP DOMAIN priv_testdomain1; -- ok +-- truncate +SET SESSION AUTHORIZATION regress_priv_user5; +TRUNCATE atest2; -- ok +TRUNCATE atest3; -- fail +ERROR: permission denied for table atest3 +-- has_table_privilege function +-- bad-input checks +select has_table_privilege(NULL,'pg_authid','select'); + has_table_privilege +--------------------- + +(1 row) + +select has_table_privilege('pg_shad','select'); +ERROR: relation "pg_shad" does not exist +select has_table_privilege('nosuchuser','pg_authid','select'); +ERROR: role "nosuchuser" does not exist +select has_table_privilege('pg_authid','sel'); +ERROR: unrecognized privilege type: "sel" +select has_table_privilege(-999999,'pg_authid','update'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(1,'select'); + has_table_privilege +--------------------- + +(1 row) + +-- superuser +\c - +select has_table_privilege(current_user,'pg_authid','select'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(current_user,'pg_authid','insert'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t2.oid,'pg_authid','update') +from (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t2.oid,'pg_authid','delete') +from (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + t +(1 row) + +-- 'rule' privilege no longer exists, but for backwards compatibility +-- has_table_privilege still recognizes the keyword and says FALSE +select has_table_privilege(current_user,t1.oid,'rule') +from (select oid from pg_class where relname = 'pg_authid') as t1; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(current_user,t1.oid,'references') +from (select oid from pg_class where relname = 'pg_authid') as t1; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t2.oid,t1.oid,'select') +from (select oid from pg_class where relname = 'pg_authid') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t2.oid,t1.oid,'insert') +from (select oid from pg_class where relname = 'pg_authid') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege('pg_authid','update'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege('pg_authid','delete'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege('pg_authid','truncate'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t1.oid,'select') +from (select oid from pg_class where relname = 'pg_authid') as t1; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t1.oid,'trigger') +from (select oid from pg_class where relname = 'pg_authid') as t1; + has_table_privilege +--------------------- + t +(1 row) + +-- non-superuser +SET SESSION AUTHORIZATION regress_priv_user3; +select has_table_privilege(current_user,'pg_class','select'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(current_user,'pg_class','insert'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t2.oid,'pg_class','update') +from (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t2.oid,'pg_class','delete') +from (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(current_user,t1.oid,'references') +from (select oid from pg_class where relname = 'pg_class') as t1; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t2.oid,t1.oid,'select') +from (select oid from pg_class where relname = 'pg_class') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t2.oid,t1.oid,'insert') +from (select oid from pg_class where relname = 'pg_class') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege('pg_class','update'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege('pg_class','delete'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege('pg_class','truncate'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t1.oid,'select') +from (select oid from pg_class where relname = 'pg_class') as t1; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t1.oid,'trigger') +from (select oid from pg_class where relname = 'pg_class') as t1; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(current_user,'atest1','select'); + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(current_user,'atest1','insert'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t2.oid,'atest1','update') +from (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t2.oid,'atest1','delete') +from (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(current_user,t1.oid,'references') +from (select oid from pg_class where relname = 'atest1') as t1; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t2.oid,t1.oid,'select') +from (select oid from pg_class where relname = 'atest1') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t2.oid,t1.oid,'insert') +from (select oid from pg_class where relname = 'atest1') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege('atest1','update'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege('atest1','delete'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege('atest1','truncate'); + has_table_privilege +--------------------- + f +(1 row) + +select has_table_privilege(t1.oid,'select') +from (select oid from pg_class where relname = 'atest1') as t1; + has_table_privilege +--------------------- + t +(1 row) + +select has_table_privilege(t1.oid,'trigger') +from (select oid from pg_class where relname = 'atest1') as t1; + has_table_privilege +--------------------- + f +(1 row) + +-- has_column_privilege function +-- bad-input checks (as non-super-user) +select has_column_privilege('pg_authid',NULL,'select'); + has_column_privilege +---------------------- + +(1 row) + +select has_column_privilege('pg_authid','nosuchcol','select'); +ERROR: column "nosuchcol" of relation "pg_authid" does not exist +select has_column_privilege(9999,'nosuchcol','select'); + has_column_privilege +---------------------- + +(1 row) + +select has_column_privilege(9999,99::int2,'select'); + has_column_privilege +---------------------- + +(1 row) + +select has_column_privilege('pg_authid',99::int2,'select'); + has_column_privilege +---------------------- + +(1 row) + +select has_column_privilege(9999,99::int2,'select'); + has_column_privilege +---------------------- + +(1 row) + +create temp table mytable(f1 int, f2 int, f3 int); +alter table mytable drop column f2; +select has_column_privilege('mytable','f2','select'); +ERROR: column "f2" of relation "mytable" does not exist +select has_column_privilege('mytable','........pg.dropped.2........','select'); + has_column_privilege +---------------------- + +(1 row) + +select has_column_privilege('mytable',2::int2,'select'); + has_column_privilege +---------------------- + t +(1 row) + +revoke select on table mytable from regress_priv_user3; +select has_column_privilege('mytable',2::int2,'select'); + has_column_privilege +---------------------- + +(1 row) + +drop table mytable; +-- Grant options +SET SESSION AUTHORIZATION regress_priv_user1; +CREATE TABLE atest4 (a int); +GRANT SELECT ON atest4 TO regress_priv_user2 WITH GRANT OPTION; +GRANT UPDATE ON atest4 TO regress_priv_user2; +GRANT SELECT ON atest4 TO GROUP regress_priv_group1 WITH GRANT OPTION; +SET SESSION AUTHORIZATION regress_priv_user2; +GRANT SELECT ON atest4 TO regress_priv_user3; +GRANT UPDATE ON atest4 TO regress_priv_user3; -- fail +WARNING: no privileges were granted for "atest4" +SET SESSION AUTHORIZATION regress_priv_user1; +REVOKE SELECT ON atest4 FROM regress_priv_user3; -- does nothing +SELECT has_table_privilege('regress_priv_user3', 'atest4', 'SELECT'); -- true + has_table_privilege +--------------------- + t +(1 row) + +REVOKE SELECT ON atest4 FROM regress_priv_user2; -- fail +ERROR: dependent privileges exist +HINT: Use CASCADE to revoke them too. +REVOKE GRANT OPTION FOR SELECT ON atest4 FROM regress_priv_user2 CASCADE; -- ok +SELECT has_table_privilege('regress_priv_user2', 'atest4', 'SELECT'); -- true + has_table_privilege +--------------------- + t +(1 row) + +SELECT has_table_privilege('regress_priv_user3', 'atest4', 'SELECT'); -- false + has_table_privilege +--------------------- + f +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'atest4', 'SELECT WITH GRANT OPTION'); -- true + has_table_privilege +--------------------- + t +(1 row) + +-- Admin options +SET SESSION AUTHORIZATION regress_priv_user4; +CREATE FUNCTION dogrant_ok() RETURNS void LANGUAGE sql SECURITY DEFINER AS + 'GRANT regress_priv_group2 TO regress_priv_user5'; +GRANT regress_priv_group2 TO regress_priv_user5; -- ok: had ADMIN OPTION +SET ROLE regress_priv_group2; +GRANT regress_priv_group2 TO regress_priv_user5; -- fails: SET ROLE suspended privilege +ERROR: must have admin option on role "regress_priv_group2" +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT regress_priv_group2 TO regress_priv_user5; -- fails: no ADMIN OPTION +ERROR: must have admin option on role "regress_priv_group2" +SELECT dogrant_ok(); -- ok: SECURITY DEFINER conveys ADMIN +NOTICE: role "regress_priv_user5" is already a member of role "regress_priv_group2" + dogrant_ok +------------ + +(1 row) + +SET ROLE regress_priv_group2; +GRANT regress_priv_group2 TO regress_priv_user5; -- fails: SET ROLE did not help +ERROR: must have admin option on role "regress_priv_group2" +SET SESSION AUTHORIZATION regress_priv_group2; +GRANT regress_priv_group2 TO regress_priv_user5; -- ok: a role can self-admin +NOTICE: role "regress_priv_user5" is already a member of role "regress_priv_group2" +CREATE FUNCTION dogrant_fails() RETURNS void LANGUAGE sql SECURITY DEFINER AS + 'GRANT regress_priv_group2 TO regress_priv_user5'; +SELECT dogrant_fails(); -- fails: no self-admin in SECURITY DEFINER +ERROR: must have admin option on role "regress_priv_group2" +CONTEXT: SQL function "dogrant_fails" statement 1 +DROP FUNCTION dogrant_fails(); +SET SESSION AUTHORIZATION regress_priv_user4; +DROP FUNCTION dogrant_ok(); +REVOKE regress_priv_group2 FROM regress_priv_user5; +-- has_sequence_privilege tests +\c - +CREATE SEQUENCE x_seq; +GRANT USAGE on x_seq to regress_priv_user2; +SELECT has_sequence_privilege('regress_priv_user1', 'atest1', 'SELECT'); +ERROR: "atest1" is not a sequence +SELECT has_sequence_privilege('regress_priv_user1', 'x_seq', 'INSERT'); +ERROR: unrecognized privilege type: "INSERT" +SELECT has_sequence_privilege('regress_priv_user1', 'x_seq', 'SELECT'); + has_sequence_privilege +------------------------ + f +(1 row) + +SET SESSION AUTHORIZATION regress_priv_user2; +SELECT has_sequence_privilege('x_seq', 'USAGE'); + has_sequence_privilege +------------------------ + t +(1 row) + +-- largeobject privilege tests +\c - +SET SESSION AUTHORIZATION regress_priv_user1; +SELECT lo_create(1001); + lo_create +----------- + 1001 +(1 row) + +SELECT lo_create(1002); + lo_create +----------- + 1002 +(1 row) + +SELECT lo_create(1003); + lo_create +----------- + 1003 +(1 row) + +SELECT lo_create(1004); + lo_create +----------- + 1004 +(1 row) + +SELECT lo_create(1005); + lo_create +----------- + 1005 +(1 row) + +GRANT ALL ON LARGE OBJECT 1001 TO PUBLIC; +GRANT SELECT ON LARGE OBJECT 1003 TO regress_priv_user2; +GRANT SELECT,UPDATE ON LARGE OBJECT 1004 TO regress_priv_user2; +GRANT ALL ON LARGE OBJECT 1005 TO regress_priv_user2; +GRANT SELECT ON LARGE OBJECT 1005 TO regress_priv_user2 WITH GRANT OPTION; +GRANT SELECT, INSERT ON LARGE OBJECT 1001 TO PUBLIC; -- to be failed +ERROR: invalid privilege type INSERT for large object +GRANT SELECT, UPDATE ON LARGE OBJECT 1001 TO nosuchuser; -- to be failed +ERROR: role "nosuchuser" does not exist +GRANT SELECT, UPDATE ON LARGE OBJECT 999 TO PUBLIC; -- to be failed +ERROR: large object 999 does not exist +\c - +SET SESSION AUTHORIZATION regress_priv_user2; +SELECT lo_create(2001); + lo_create +----------- + 2001 +(1 row) + +SELECT lo_create(2002); + lo_create +----------- + 2002 +(1 row) + +SELECT loread(lo_open(1001, x'20000'::int), 32); -- allowed, for now + loread +-------- + \x +(1 row) + +SELECT lowrite(lo_open(1001, x'40000'::int), 'abcd'); -- fail, wrong mode +ERROR: large object descriptor 0 was not opened for writing +SELECT loread(lo_open(1001, x'40000'::int), 32); + loread +-------- + \x +(1 row) + +SELECT loread(lo_open(1002, x'40000'::int), 32); -- to be denied +ERROR: permission denied for large object 1002 +SELECT loread(lo_open(1003, x'40000'::int), 32); + loread +-------- + \x +(1 row) + +SELECT loread(lo_open(1004, x'40000'::int), 32); + loread +-------- + \x +(1 row) + +SELECT lowrite(lo_open(1001, x'20000'::int), 'abcd'); + lowrite +--------- + 4 +(1 row) + +SELECT lowrite(lo_open(1002, x'20000'::int), 'abcd'); -- to be denied +ERROR: permission denied for large object 1002 +SELECT lowrite(lo_open(1003, x'20000'::int), 'abcd'); -- to be denied +ERROR: permission denied for large object 1003 +SELECT lowrite(lo_open(1004, x'20000'::int), 'abcd'); + lowrite +--------- + 4 +(1 row) + +GRANT SELECT ON LARGE OBJECT 1005 TO regress_priv_user3; +GRANT UPDATE ON LARGE OBJECT 1006 TO regress_priv_user3; -- to be denied +ERROR: large object 1006 does not exist +REVOKE ALL ON LARGE OBJECT 2001, 2002 FROM PUBLIC; +GRANT ALL ON LARGE OBJECT 2001 TO regress_priv_user3; +SELECT lo_unlink(1001); -- to be denied +ERROR: must be owner of large object 1001 +SELECT lo_unlink(2002); + lo_unlink +----------- + 1 +(1 row) + +\c - +-- confirm ACL setting +SELECT oid, pg_get_userbyid(lomowner) ownername, lomacl FROM pg_largeobject_metadata WHERE oid >= 1000 AND oid < 3000 ORDER BY oid; + oid | ownername | lomacl +------+--------------------+------------------------------------------------------------------------------------------------------------------------------ + 1001 | regress_priv_user1 | {regress_priv_user1=rw/regress_priv_user1,=rw/regress_priv_user1} + 1002 | regress_priv_user1 | + 1003 | regress_priv_user1 | {regress_priv_user1=rw/regress_priv_user1,regress_priv_user2=r/regress_priv_user1} + 1004 | regress_priv_user1 | {regress_priv_user1=rw/regress_priv_user1,regress_priv_user2=rw/regress_priv_user1} + 1005 | regress_priv_user1 | {regress_priv_user1=rw/regress_priv_user1,regress_priv_user2=r*w/regress_priv_user1,regress_priv_user3=r/regress_priv_user2} + 2001 | regress_priv_user2 | {regress_priv_user2=rw/regress_priv_user2,regress_priv_user3=rw/regress_priv_user2} +(6 rows) + +SET SESSION AUTHORIZATION regress_priv_user3; +SELECT loread(lo_open(1001, x'40000'::int), 32); + loread +------------ + \x61626364 +(1 row) + +SELECT loread(lo_open(1003, x'40000'::int), 32); -- to be denied +ERROR: permission denied for large object 1003 +SELECT loread(lo_open(1005, x'40000'::int), 32); + loread +-------- + \x +(1 row) + +SELECT lo_truncate(lo_open(1005, x'20000'::int), 10); -- to be denied +ERROR: permission denied for large object 1005 +SELECT lo_truncate(lo_open(2001, x'20000'::int), 10); + lo_truncate +------------- + 0 +(1 row) + +-- compatibility mode in largeobject permission +\c - +SET lo_compat_privileges = false; -- default setting +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT loread(lo_open(1002, x'40000'::int), 32); -- to be denied +ERROR: permission denied for large object 1002 +SELECT lowrite(lo_open(1002, x'20000'::int), 'abcd'); -- to be denied +ERROR: permission denied for large object 1002 +SELECT lo_truncate(lo_open(1002, x'20000'::int), 10); -- to be denied +ERROR: permission denied for large object 1002 +SELECT lo_put(1002, 1, 'abcd'); -- to be denied +ERROR: permission denied for large object 1002 +SELECT lo_unlink(1002); -- to be denied +ERROR: must be owner of large object 1002 +SELECT lo_export(1001, '/dev/null'); -- to be denied +ERROR: permission denied for function lo_export +SELECT lo_import('/dev/null'); -- to be denied +ERROR: permission denied for function lo_import +SELECT lo_import('/dev/null', 2003); -- to be denied +ERROR: permission denied for function lo_import +\c - +SET lo_compat_privileges = true; -- compatibility mode +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT loread(lo_open(1002, x'40000'::int), 32); + loread +-------- + \x +(1 row) + +SELECT lowrite(lo_open(1002, x'20000'::int), 'abcd'); + lowrite +--------- + 4 +(1 row) + +SELECT lo_truncate(lo_open(1002, x'20000'::int), 10); + lo_truncate +------------- + 0 +(1 row) + +SELECT lo_unlink(1002); + lo_unlink +----------- + 1 +(1 row) + +SELECT lo_export(1001, '/dev/null'); -- to be denied +ERROR: permission denied for function lo_export +-- don't allow unpriv users to access pg_largeobject contents +\c - +SELECT * FROM pg_largeobject LIMIT 0; + loid | pageno | data +------+--------+------ +(0 rows) + +SET SESSION AUTHORIZATION regress_priv_user1; +SELECT * FROM pg_largeobject LIMIT 0; -- to be denied +ERROR: permission denied for table pg_largeobject +-- test default ACLs +\c - +CREATE SCHEMA testns; +GRANT ALL ON SCHEMA testns TO regress_priv_user1; +CREATE TABLE testns.acltest1 (x int); +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- no + has_table_privilege +--------------------- + f +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + has_table_privilege +--------------------- + f +(1 row) + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT SELECT ON TABLES TO public; +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- no + has_table_privilege +--------------------- + f +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + has_table_privilege +--------------------- + f +(1 row) + +DROP TABLE testns.acltest1; +CREATE TABLE testns.acltest1 (x int); +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- yes + has_table_privilege +--------------------- + t +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + has_table_privilege +--------------------- + f +(1 row) + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT INSERT ON TABLES TO regress_priv_user1; +DROP TABLE testns.acltest1; +CREATE TABLE testns.acltest1 (x int); +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- yes + has_table_privilege +--------------------- + t +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- yes + has_table_privilege +--------------------- + t +(1 row) + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns REVOKE INSERT ON TABLES FROM regress_priv_user1; +DROP TABLE testns.acltest1; +CREATE TABLE testns.acltest1 (x int); +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- yes + has_table_privilege +--------------------- + t +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + has_table_privilege +--------------------- + f +(1 row) + +ALTER DEFAULT PRIVILEGES FOR ROLE regress_priv_user1 REVOKE EXECUTE ON FUNCTIONS FROM public; +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT USAGE ON SCHEMAS TO regress_priv_user2; -- error +ERROR: cannot use IN SCHEMA clause when using GRANT/REVOKE ON SCHEMAS +-- +-- Testing blanket default grants is very hazardous since it might change +-- the privileges attached to objects created by concurrent regression tests. +-- To avoid that, be sure to revoke the privileges again before committing. +-- +BEGIN; +ALTER DEFAULT PRIVILEGES GRANT USAGE ON SCHEMAS TO regress_priv_user2; +CREATE SCHEMA testns2; +SELECT has_schema_privilege('regress_priv_user2', 'testns2', 'USAGE'); -- yes + has_schema_privilege +---------------------- + t +(1 row) + +SELECT has_schema_privilege('regress_priv_user2', 'testns2', 'CREATE'); -- no + has_schema_privilege +---------------------- + f +(1 row) + +ALTER DEFAULT PRIVILEGES REVOKE USAGE ON SCHEMAS FROM regress_priv_user2; +CREATE SCHEMA testns3; +SELECT has_schema_privilege('regress_priv_user2', 'testns3', 'USAGE'); -- no + has_schema_privilege +---------------------- + f +(1 row) + +SELECT has_schema_privilege('regress_priv_user2', 'testns3', 'CREATE'); -- no + has_schema_privilege +---------------------- + f +(1 row) + +ALTER DEFAULT PRIVILEGES GRANT ALL ON SCHEMAS TO regress_priv_user2; +CREATE SCHEMA testns4; +SELECT has_schema_privilege('regress_priv_user2', 'testns4', 'USAGE'); -- yes + has_schema_privilege +---------------------- + t +(1 row) + +SELECT has_schema_privilege('regress_priv_user2', 'testns4', 'CREATE'); -- yes + has_schema_privilege +---------------------- + t +(1 row) + +ALTER DEFAULT PRIVILEGES REVOKE ALL ON SCHEMAS FROM regress_priv_user2; +COMMIT; +CREATE SCHEMA testns5; +SELECT has_schema_privilege('regress_priv_user2', 'testns5', 'USAGE'); -- no + has_schema_privilege +---------------------- + f +(1 row) + +SELECT has_schema_privilege('regress_priv_user2', 'testns5', 'CREATE'); -- no + has_schema_privilege +---------------------- + f +(1 row) + +SET ROLE regress_priv_user1; +CREATE FUNCTION testns.foo() RETURNS int AS 'select 1' LANGUAGE sql; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE AGGREGATE testns.agg1(int) (sfunc = int4pl, stype = int4); +ERROR: CREATE AGGREGATE not supported yet +LINE 1: CREATE AGGREGATE testns.agg1(int) (sfunc = int4pl, stype = i... + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- +CREATE PROCEDURE testns.bar() AS 'select 1' LANGUAGE sql; +SELECT has_function_privilege('regress_priv_user2', 'testns.foo()', 'EXECUTE'); -- no + has_function_privilege +------------------------ + f +(1 row) + +-- SELECT has_function_privilege('regress_priv_user2', 'testns.agg1(int)', 'EXECUTE'); -- no +SELECT has_function_privilege('regress_priv_user2', 'testns.bar()', 'EXECUTE'); -- no + has_function_privilege +------------------------ + f +(1 row) + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT EXECUTE ON ROUTINES to public; +DROP FUNCTION testns.foo(); +CREATE FUNCTION testns.foo() RETURNS int AS 'select 1' LANGUAGE sql; +-- DROP AGGREGATE testns.agg1(int); +-- CREATE AGGREGATE testns.agg1(int) (sfunc = int4pl, stype = int4); +DROP PROCEDURE testns.bar(); +CREATE PROCEDURE testns.bar() AS 'select 1' LANGUAGE sql; +SELECT has_function_privilege('regress_priv_user2', 'testns.foo()', 'EXECUTE'); -- yes + has_function_privilege +------------------------ + t +(1 row) + +-- SELECT has_function_privilege('regress_priv_user2', 'testns.agg1(int)', 'EXECUTE'); -- yes +SELECT has_function_privilege('regress_priv_user2', 'testns.bar()', 'EXECUTE'); -- yes (counts as function here) + has_function_privilege +------------------------ + t +(1 row) + +DROP FUNCTION testns.foo(); +-- DROP AGGREGATE testns.agg1(int); +DROP PROCEDURE testns.bar(); +ALTER DEFAULT PRIVILEGES FOR ROLE regress_priv_user1 REVOKE USAGE ON TYPES FROM public; +CREATE DOMAIN testns.priv_testdomain1 AS int; +SELECT has_type_privilege('regress_priv_user2', 'testns.priv_testdomain1', 'USAGE'); -- no + has_type_privilege +-------------------- + f +(1 row) + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT USAGE ON TYPES to public; +DROP DOMAIN testns.priv_testdomain1; +CREATE DOMAIN testns.priv_testdomain1 AS int; +SELECT has_type_privilege('regress_priv_user2', 'testns.priv_testdomain1', 'USAGE'); -- yes + has_type_privilege +-------------------- + t +(1 row) + +DROP DOMAIN testns.priv_testdomain1; +RESET ROLE; +SELECT count(*) + FROM pg_default_acl d LEFT JOIN pg_namespace n ON defaclnamespace = n.oid + WHERE nspname = 'testns'; + count +------- + 3 +(1 row) + +DROP SCHEMA testns CASCADE; +NOTICE: drop cascades to table testns.acltest1 +DROP SCHEMA testns2 CASCADE; +DROP SCHEMA testns4 CASCADE; +DROP SCHEMA testns5 CASCADE; +SELECT d.* -- check that entries went away + FROM pg_default_acl d LEFT JOIN pg_namespace n ON defaclnamespace = n.oid + WHERE nspname IS NULL AND defaclnamespace != 0; + defaclrole | defaclnamespace | defaclobjtype | defaclacl +------------+-----------------+---------------+----------- +(0 rows) + +-- Grant on all objects of given type in a schema +\c - +CREATE SCHEMA testns; +CREATE TABLE testns.t1 (f1 int); +CREATE TABLE testns.t2 (f1 int); +SELECT has_table_privilege('regress_priv_user1', 'testns.t1', 'SELECT'); -- false + has_table_privilege +--------------------- + f +(1 row) + +GRANT ALL ON ALL TABLES IN SCHEMA testns TO regress_priv_user1; +SELECT has_table_privilege('regress_priv_user1', 'testns.t1', 'SELECT'); -- true + has_table_privilege +--------------------- + t +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.t2', 'SELECT'); -- true + has_table_privilege +--------------------- + t +(1 row) + +REVOKE ALL ON ALL TABLES IN SCHEMA testns FROM regress_priv_user1; +SELECT has_table_privilege('regress_priv_user1', 'testns.t1', 'SELECT'); -- false + has_table_privilege +--------------------- + f +(1 row) + +SELECT has_table_privilege('regress_priv_user1', 'testns.t2', 'SELECT'); -- false + has_table_privilege +--------------------- + f +(1 row) + +CREATE FUNCTION testns.priv_testfunc(int) RETURNS int AS 'select 3 * $1;' LANGUAGE sql; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE AGGREGATE testns.priv_testagg(int) (sfunc = int4pl, stype = int4); +ERROR: CREATE AGGREGATE not supported yet +LINE 1: CREATE AGGREGATE testns.priv_testagg(int) (sfunc = int4pl, s... + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- +CREATE PROCEDURE testns.priv_testproc(int) AS 'select 3' LANGUAGE sql; +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testfunc(int)', 'EXECUTE'); -- true by default + has_function_privilege +------------------------ + t +(1 row) + +-- SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testagg(int)', 'EXECUTE'); -- true by default +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- true by default + has_function_privilege +------------------------ + t +(1 row) + +REVOKE ALL ON ALL FUNCTIONS IN SCHEMA testns FROM PUBLIC; +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testfunc(int)', 'EXECUTE'); -- false + has_function_privilege +------------------------ + f +(1 row) + +-- SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testagg(int)', 'EXECUTE'); -- false +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- still true, not a function + has_function_privilege +------------------------ + t +(1 row) + +REVOKE ALL ON ALL PROCEDURES IN SCHEMA testns FROM PUBLIC; +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- now false + has_function_privilege +------------------------ + f +(1 row) + +GRANT ALL ON ALL ROUTINES IN SCHEMA testns TO PUBLIC; +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testfunc(int)', 'EXECUTE'); -- true + has_function_privilege +------------------------ + t +(1 row) + +-- SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testagg(int)', 'EXECUTE'); -- true +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- true + has_function_privilege +------------------------ + t +(1 row) + +\set VERBOSITY terse \\ -- suppress cascade details +DROP SCHEMA testns CASCADE; +NOTICE: drop cascades to 4 other objects +\set VERBOSITY default +-- Change owner of the schema & and rename of new schema owner +\c - +CREATE ROLE regress_schemauser1 superuser login; +CREATE ROLE regress_schemauser2 superuser login; +SET SESSION ROLE regress_schemauser1; +CREATE SCHEMA testns; +SELECT nspname, rolname FROM pg_namespace, pg_roles WHERE pg_namespace.nspname = 'testns' AND pg_namespace.nspowner = pg_roles.oid; + nspname | rolname +---------+--------------------- + testns | regress_schemauser1 +(1 row) + +ALTER SCHEMA testns OWNER TO regress_schemauser2; +ALTER ROLE regress_schemauser2 RENAME TO regress_schemauser_renamed; +SELECT nspname, rolname FROM pg_namespace, pg_roles WHERE pg_namespace.nspname = 'testns' AND pg_namespace.nspowner = pg_roles.oid; + nspname | rolname +---------+---------------------------- + testns | regress_schemauser_renamed +(1 row) + +set session role regress_schemauser_renamed; +\set VERBOSITY terse \\ -- suppress cascade details +DROP SCHEMA testns CASCADE; +\set VERBOSITY default +-- clean up +\c - +DROP ROLE regress_schemauser1; +DROP ROLE regress_schemauser_renamed; +-- test that dependent privileges are revoked (or not) properly +\c - +set session role regress_priv_user1; +create table dep_priv_test (a int); +grant select on dep_priv_test to regress_priv_user2 with grant option; +grant select on dep_priv_test to regress_priv_user3 with grant option; +set session role regress_priv_user2; +grant select on dep_priv_test to regress_priv_user4 with grant option; +set session role regress_priv_user3; +grant select on dep_priv_test to regress_priv_user4 with grant option; +set session role regress_priv_user4; +grant select on dep_priv_test to regress_priv_user5; +\dp dep_priv_test + Access privileges + Schema | Name | Type | Access privileges | Column privileges | Policies +--------+---------------+-------+-----------------------------------------------+-------------------+---------- + public | dep_priv_test | table | regress_priv_user1=arwdDxt/regress_priv_user1+| | + | | | regress_priv_user2=r*/regress_priv_user1 +| | + | | | regress_priv_user3=r*/regress_priv_user1 +| | + | | | regress_priv_user4=r*/regress_priv_user2 +| | + | | | regress_priv_user4=r*/regress_priv_user3 +| | + | | | regress_priv_user5=r/regress_priv_user4 | | +(1 row) + +set session role regress_priv_user2; +revoke select on dep_priv_test from regress_priv_user4 cascade; +\dp dep_priv_test + Access privileges + Schema | Name | Type | Access privileges | Column privileges | Policies +--------+---------------+-------+-----------------------------------------------+-------------------+---------- + public | dep_priv_test | table | regress_priv_user1=arwdDxt/regress_priv_user1+| | + | | | regress_priv_user2=r*/regress_priv_user1 +| | + | | | regress_priv_user3=r*/regress_priv_user1 +| | + | | | regress_priv_user4=r*/regress_priv_user3 +| | + | | | regress_priv_user5=r/regress_priv_user4 | | +(1 row) + +set session role regress_priv_user3; +revoke select on dep_priv_test from regress_priv_user4 cascade; +\dp dep_priv_test + Access privileges + Schema | Name | Type | Access privileges | Column privileges | Policies +--------+---------------+-------+-----------------------------------------------+-------------------+---------- + public | dep_priv_test | table | regress_priv_user1=arwdDxt/regress_priv_user1+| | + | | | regress_priv_user2=r*/regress_priv_user1 +| | + | | | regress_priv_user3=r*/regress_priv_user1 | | +(1 row) + +set session role regress_priv_user1; +drop table dep_priv_test; +-- clean up +\c +drop sequence x_seq; +-- DROP AGGREGATE priv_testagg1(int); +DROP FUNCTION priv_testfunc2(int); +DROP FUNCTION priv_testfunc4(boolean); +DROP PROCEDURE priv_testproc1(int); +DROP VIEW atestv0; +DROP VIEW atestv1; +DROP VIEW atestv2; +-- this should cascade to drop atestv4 +DROP VIEW atestv3 CASCADE; +NOTICE: drop cascades to view atestv4 +-- this should complain "does not exist" +DROP VIEW atestv4; +ERROR: view "atestv4" does not exist +DROP TABLE atest1; +DROP TABLE atest2; +DROP TABLE atest3; +DROP TABLE atest4; +DROP TABLE atest5; +DROP TABLE atest6; +DROP TABLE priv_int8_tbl; +DROP SEQUENCE twoseq; +DROP SEQUENCE fourseq; +SELECT lo_unlink(oid) FROM pg_largeobject_metadata WHERE oid >= 1000 AND oid < 3000 ORDER BY oid; + lo_unlink +----------- + 1 + 1 + 1 + 1 + 1 +(5 rows) + +DROP GROUP regress_priv_group1; +DROP GROUP regress_priv_group2; +-- these are needed to clean up permissions +REVOKE USAGE ON LANGUAGE sql FROM regress_priv_user1; +DROP OWNED BY regress_priv_user1; +DROP USER regress_priv_user1; +DROP USER regress_priv_user2; +DROP USER regress_priv_user3; +DROP USER regress_priv_user4; +DROP USER regress_priv_user5; +DROP USER regress_priv_user6; +ERROR: role "regress_priv_user6" does not exist +-- NOT SUPPORTED +-- +-- -- permissions with LOCK TABLE +-- CREATE USER regress_locktable_user; +-- CREATE TABLE lock_table (a int); +-- +-- -- LOCK TABLE and SELECT permission +-- GRANT SELECT ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should fail +ERROR: ROW EXCLUSIVE not supported yet +LINE 1: LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- ROLLBACK; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS SHARE MODE; -- should pass +-- COMMIT; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should fail +-- ROLLBACK; +-- \c +-- REVOKE SELECT ON lock_table FROM regress_locktable_user; +-- +-- -- LOCK TABLE and INSERT permission +-- GRANT INSERT ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- BEGIN;BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- \c +-- REVOKE UPDATE ON lock_table FROM regress_locktable_user; +-- +-- -- LOCK TABLE and DELETE permission +-- GRANT DELETE ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS SHARE MODE; -- should fail +-- ROLLBACK; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- \c +-- REVOKE DELETE ON lock_table FROM regress_locktable_user; +-- +-- -- LOCK TABLE and TRUNCATE permission +-- GRANT TRUNCATE ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS SHARE MODE; -- should fail +-- ROLLBACK; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- \c +-- REVOKE TRUNCATE ON lock_table FROM regress_locktable_user; +-- +-- -- clean up +-- DROP TABLE lock_table; +-- DROP USER regress_locktable_user; +-- diff --git a/src/postgres/src/test/regress/expected/yb_roleattributes.out b/src/postgres/src/test/regress/expected/yb_roleattributes.out new file mode 100644 index 000000000000..570aa5f8343f --- /dev/null +++ b/src/postgres/src/test/regress/expected/yb_roleattributes.out @@ -0,0 +1,249 @@ +-- default for superuser is false +CREATE ROLE regress_test_def_superuser; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_superuser'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_superuser | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_superuser WITH SUPERUSER; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_superuser'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_superuser | t | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_superuser WITH NOSUPERUSER; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_superuser'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_superuser | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_superuser WITH SUPERUSER; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_superuser'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_superuser | t | t | f | f | f | f | f | -1 | | +(1 row) + +-- default for inherit is true +CREATE ROLE regress_test_def_inherit; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_inherit'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +--------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_inherit | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_inherit WITH NOINHERIT; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_inherit'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_inherit | f | f | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_inherit WITH INHERIT; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_inherit'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_inherit | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_inherit WITH NOINHERIT; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_inherit'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_inherit | f | f | f | f | f | f | f | -1 | | +(1 row) + +-- default for create role is false +CREATE ROLE regress_test_def_createrole; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_createrole'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_createrole | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_createrole WITH CREATEROLE; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createrole'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_createrole | f | t | t | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_createrole WITH NOCREATEROLE; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createrole'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_createrole | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_createrole WITH CREATEROLE; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createrole'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_createrole | f | t | t | f | f | f | f | -1 | | +(1 row) + +-- default for create database is false +CREATE ROLE regress_test_def_createdb; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_createdb'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +---------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_createdb | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_createdb WITH CREATEDB; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createdb'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-----------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_createdb | f | t | f | t | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_createdb WITH NOCREATEDB; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createdb'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-----------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_createdb | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_createdb WITH CREATEDB; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createdb'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +-----------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_createdb | f | t | f | t | f | f | f | -1 | | +(1 row) + +-- default for can login is false for role +CREATE ROLE regress_test_def_role_canlogin; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_role_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +--------------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_role_canlogin | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_role_canlogin WITH LOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_role_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_role_canlogin | f | t | f | f | t | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_role_canlogin WITH NOLOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_role_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_role_canlogin | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_role_canlogin WITH LOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_role_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_role_canlogin | f | t | f | f | t | f | f | -1 | | +(1 row) + +-- default for can login is true for user +CREATE USER regress_test_def_user_canlogin; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_user_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +--------------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_user_canlogin | f | t | f | f | t | f | f | -1 | | +(1 row) + +CREATE USER regress_test_user_canlogin WITH NOLOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_user_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_user_canlogin | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER USER regress_test_user_canlogin WITH LOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_user_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_user_canlogin | f | t | f | f | t | f | f | -1 | | +(1 row) + +ALTER USER regress_test_user_canlogin WITH NOLOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_user_canlogin'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_user_canlogin | f | t | f | f | f | f | f | -1 | | +(1 row) + +-- default for replication is false +CREATE ROLE regress_test_def_replication; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_replication'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_replication | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_replication WITH REPLICATION; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_replication'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +--------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_replication | f | t | f | f | f | t | f | -1 | | +(1 row) + +ALTER ROLE regress_test_replication WITH NOREPLICATION; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_replication'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +--------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_replication | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_replication WITH REPLICATION; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_replication'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +--------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_replication | f | t | f | f | f | t | f | -1 | | +(1 row) + +-- default for bypassrls is false +CREATE ROLE regress_test_def_bypassrls; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_bypassrls'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +----------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_def_bypassrls | f | t | f | f | f | f | f | -1 | | +(1 row) + +CREATE ROLE regress_test_bypassrls WITH BYPASSRLS; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_bypassrls'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_bypassrls | f | t | f | f | f | f | t | -1 | | +(1 row) + +ALTER ROLE regress_test_bypassrls WITH NOBYPASSRLS; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_bypassrls'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_bypassrls | f | t | f | f | f | f | f | -1 | | +(1 row) + +ALTER ROLE regress_test_bypassrls WITH BYPASSRLS; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_bypassrls'; + rolname | rolsuper | rolinherit | rolcreaterole | rolcreatedb | rolcanlogin | rolreplication | rolbypassrls | rolconnlimit | rolpassword | rolvaliduntil +------------------------+----------+------------+---------------+-------------+-------------+----------------+--------------+--------------+-------------+--------------- + regress_test_bypassrls | f | t | f | f | f | f | t | -1 | | +(1 row) + +-- clean up roles +DROP ROLE regress_test_def_superuser; +DROP ROLE regress_test_superuser; +DROP ROLE regress_test_def_inherit; +DROP ROLE regress_test_inherit; +DROP ROLE regress_test_def_createrole; +DROP ROLE regress_test_createrole; +DROP ROLE regress_test_def_createdb; +DROP ROLE regress_test_createdb; +DROP ROLE regress_test_def_role_canlogin; +DROP ROLE regress_test_role_canlogin; +DROP USER regress_test_def_user_canlogin; +DROP USER regress_test_user_canlogin; +DROP ROLE regress_test_def_replication; +DROP ROLE regress_test_replication; +DROP ROLE regress_test_def_bypassrls; +DROP ROLE regress_test_bypassrls; diff --git a/src/postgres/src/test/regress/expected/yb_rolenames.out b/src/postgres/src/test/regress/expected/yb_rolenames.out new file mode 100644 index 000000000000..5c99ba657c71 --- /dev/null +++ b/src/postgres/src/test/regress/expected/yb_rolenames.out @@ -0,0 +1,859 @@ +CREATE OR REPLACE FUNCTION chkrolattr() + RETURNS TABLE ("role" name, rolekeyword text, canlogin bool, replication bool) + AS $$ +SELECT r.rolname, v.keyword, r.rolcanlogin, r.rolreplication + FROM pg_roles r + JOIN (VALUES(CURRENT_USER, 'current_user'), + (SESSION_USER, 'session_user'), + ('current_user', '-'), + ('session_user', '-'), + ('Public', '-'), + ('None', '-')) + AS v(uname, keyword) + ON (r.rolname = v.uname) + ORDER BY 1; +$$ LANGUAGE SQL; +CREATE OR REPLACE FUNCTION chksetconfig() + RETURNS TABLE (db name, "role" name, rolkeyword text, setconfig text[]) + AS $$ +SELECT COALESCE(d.datname, 'ALL'), COALESCE(r.rolname, 'ALL'), + COALESCE(v.keyword, '-'), s.setconfig + FROM pg_db_role_setting s + LEFT JOIN pg_roles r ON (r.oid = s.setrole) + LEFT JOIN pg_database d ON (d.oid = s.setdatabase) + LEFT JOIN (VALUES(CURRENT_USER, 'current_user'), + (SESSION_USER, 'session_user')) + AS v(uname, keyword) + ON (r.rolname = v.uname) + WHERE (r.rolname) IN ('Public', 'current_user', 'regress_testrol1', 'regress_testrol2') +ORDER BY 1, 2; +$$ LANGUAGE SQL; +CREATE OR REPLACE FUNCTION chkumapping() + RETURNS TABLE (umname name, umserver name, umoptions text[]) + AS $$ +SELECT r.rolname, s.srvname, m.umoptions + FROM pg_user_mapping m + LEFT JOIN pg_roles r ON (r.oid = m.umuser) + JOIN pg_foreign_server s ON (s.oid = m.umserver) + ORDER BY 2; +$$ LANGUAGE SQL; +CREATE ROLE "Public"; +CREATE ROLE "None"; +CREATE ROLE "current_user"; +CREATE ROLE "session_user"; +CREATE ROLE "user"; +CREATE ROLE current_user; -- error +ERROR: CURRENT_USER cannot be used as a role name here +LINE 1: CREATE ROLE current_user; + ^ +CREATE ROLE current_role; -- error +ERROR: syntax error at or near "current_role" +LINE 1: CREATE ROLE current_role; + ^ +CREATE ROLE session_user; -- error +ERROR: SESSION_USER cannot be used as a role name here +LINE 1: CREATE ROLE session_user; + ^ +CREATE ROLE user; -- error +ERROR: syntax error at or near "user" +LINE 1: CREATE ROLE user; + ^ +CREATE ROLE all; -- error +ERROR: syntax error at or near "all" +LINE 1: CREATE ROLE all; + ^ +CREATE ROLE public; -- error +ERROR: role name "public" is reserved +LINE 1: CREATE ROLE public; + ^ +CREATE ROLE "public"; -- error +ERROR: role name "public" is reserved +LINE 1: CREATE ROLE "public"; + ^ +CREATE ROLE none; -- error +ERROR: role name "none" is reserved +LINE 1: CREATE ROLE none; + ^ +CREATE ROLE "none"; -- error +ERROR: role name "none" is reserved +LINE 1: CREATE ROLE "none"; + ^ +CREATE ROLE pg_abc; -- error +ERROR: role name "pg_abc" is reserved +DETAIL: Role names starting with "pg_" are reserved. +CREATE ROLE "pg_abc"; -- error +ERROR: role name "pg_abc" is reserved +DETAIL: Role names starting with "pg_" are reserved. +CREATE ROLE pg_abcdef; -- error +ERROR: role name "pg_abcdef" is reserved +DETAIL: Role names starting with "pg_" are reserved. +CREATE ROLE "pg_abcdef"; -- error +ERROR: role name "pg_abcdef" is reserved +DETAIL: Role names starting with "pg_" are reserved. +CREATE ROLE regress_testrol0 SUPERUSER LOGIN; +CREATE ROLE regress_testrolx SUPERUSER LOGIN; +CREATE ROLE regress_testrol2 SUPERUSER; +CREATE ROLE regress_testrol1 SUPERUSER LOGIN IN ROLE regress_testrol2; +\c - +SET SESSION AUTHORIZATION regress_testrol1; +SET ROLE regress_testrol2; +-- ALTER ROLE +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | f + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | f + session_user | - | f | f +(6 rows) + +ALTER ROLE CURRENT_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | f + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | t + session_user | - | f | f +(6 rows) + +ALTER ROLE "current_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | t + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | t + session_user | - | f | f +(6 rows) + +ALTER ROLE SESSION_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | t + regress_testrol1 | session_user | t | t + regress_testrol2 | current_user | f | t + session_user | - | f | f +(6 rows) + +ALTER ROLE "session_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | t + regress_testrol1 | session_user | t | t + regress_testrol2 | current_user | f | t + session_user | - | f | t +(6 rows) + +ALTER USER "Public" WITH REPLICATION; +ALTER USER "None" WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | t + Public | - | f | t + current_user | - | f | t + regress_testrol1 | session_user | t | t + regress_testrol2 | current_user | f | t + session_user | - | f | t +(6 rows) + +ALTER USER regress_testrol1 WITH NOREPLICATION; +ALTER USER regress_testrol2 WITH NOREPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | t + Public | - | f | t + current_user | - | f | t + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | f + session_user | - | f | t +(6 rows) + +-- Manually rollback the above changes +-- TODO put this in a transaction after #1383 +ALTER ROLE "None" NOREPLICATION; +ALTER ROLE "Public" NOREPLICATION; +ALTER ROLE "current_user" NOREPLICATION; +ALTER ROLE "session_user" NOREPLICATION; +ALTER ROLE USER WITH LOGIN; -- error +ERROR: syntax error at or near "USER" +LINE 1: ALTER ROLE USER WITH LOGIN; + ^ +ALTER ROLE CURRENT_ROLE WITH LOGIN; --error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ALTER ROLE CURRENT_ROLE WITH LOGIN; + ^ +ALTER ROLE ALL WITH REPLICATION; -- error +ERROR: syntax error at or near "WITH" +LINE 1: ALTER ROLE ALL WITH REPLICATION; + ^ +ALTER ROLE SESSION_ROLE WITH NOREPLICATION; -- error +ERROR: role "session_role" does not exist +ALTER ROLE PUBLIC WITH NOREPLICATION; -- error +ERROR: role "public" does not exist +ALTER ROLE "public" WITH NOREPLICATION; -- error +ERROR: role "public" does not exist +ALTER ROLE NONE WITH NOREPLICATION; -- error +ERROR: role name "none" is reserved +LINE 1: ALTER ROLE NONE WITH NOREPLICATION; + ^ +ALTER ROLE "none" WITH NOREPLICATION; -- error +ERROR: role name "none" is reserved +LINE 1: ALTER ROLE "none" WITH NOREPLICATION; + ^ +ALTER ROLE nonexistent WITH NOREPLICATION; -- error +ERROR: role "nonexistent" does not exist +-- ALTER USER +BEGIN; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | f + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | f + session_user | - | f | f +(6 rows) + +ALTER USER CURRENT_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | f + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | t + session_user | - | f | f +(6 rows) + +ALTER USER "current_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | t + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | t + session_user | - | f | f +(6 rows) + +ALTER USER SESSION_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | t + regress_testrol1 | session_user | t | t + regress_testrol2 | current_user | f | t + session_user | - | f | f +(6 rows) + +ALTER USER "session_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | f + Public | - | f | f + current_user | - | f | t + regress_testrol1 | session_user | t | t + regress_testrol2 | current_user | f | t + session_user | - | f | t +(6 rows) + +ALTER USER "Public" WITH REPLICATION; +ALTER USER "None" WITH REPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | t + Public | - | f | t + current_user | - | f | t + regress_testrol1 | session_user | t | t + regress_testrol2 | current_user | f | t + session_user | - | f | t +(6 rows) + +ALTER USER regress_testrol1 WITH NOREPLICATION; +ALTER USER regress_testrol2 WITH NOREPLICATION; +SELECT * FROM chkrolattr(); + role | rolekeyword | canlogin | replication +------------------+--------------+----------+------------- + None | - | f | t + Public | - | f | t + current_user | - | f | t + regress_testrol1 | session_user | t | f + regress_testrol2 | current_user | f | f + session_user | - | f | t +(6 rows) + +ROLLBACK; +ALTER USER USER WITH LOGIN; -- error +ERROR: syntax error at or near "USER" +LINE 1: ALTER USER USER WITH LOGIN; + ^ +ALTER USER CURRENT_ROLE WITH LOGIN; -- error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ALTER USER CURRENT_ROLE WITH LOGIN; + ^ +ALTER USER ALL WITH REPLICATION; -- error +ERROR: syntax error at or near "WITH" +LINE 1: ALTER USER ALL WITH REPLICATION; + ^ +ALTER USER SESSION_ROLE WITH NOREPLICATION; -- error +ERROR: role "session_role" does not exist +ALTER USER PUBLIC WITH NOREPLICATION; -- error +ERROR: role "public" does not exist +ALTER USER "public" WITH NOREPLICATION; -- error +ERROR: role "public" does not exist +ALTER USER NONE WITH NOREPLICATION; -- error +ERROR: role name "none" is reserved +LINE 1: ALTER USER NONE WITH NOREPLICATION; + ^ +ALTER USER "none" WITH NOREPLICATION; -- error +ERROR: role name "none" is reserved +LINE 1: ALTER USER "none" WITH NOREPLICATION; + ^ +ALTER USER nonexistent WITH NOREPLICATION; -- error +ERROR: role "nonexistent" does not exist +-- ALTER ROLE SET/RESET +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +----+------+------------+----------- +(0 rows) + +ALTER ROLE CURRENT_USER SET application_name to 'FOO'; +ALTER ROLE SESSION_USER SET application_name to 'BAR'; +ALTER ROLE "current_user" SET application_name to 'FOOFOO'; +ALTER ROLE "Public" SET application_name to 'BARBAR'; +ALTER ROLE ALL SET application_name to 'SLAP'; +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +-----+------------------+--------------+--------------------------- + ALL | Public | - | {application_name=BARBAR} + ALL | current_user | - | {application_name=FOOFOO} + ALL | regress_testrol1 | session_user | {application_name=BAR} + ALL | regress_testrol2 | current_user | {application_name=FOO} +(4 rows) + +ALTER ROLE regress_testrol1 SET application_name to 'SLAM'; +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +-----+------------------+--------------+--------------------------- + ALL | Public | - | {application_name=BARBAR} + ALL | current_user | - | {application_name=FOOFOO} + ALL | regress_testrol1 | session_user | {application_name=SLAM} + ALL | regress_testrol2 | current_user | {application_name=FOO} +(4 rows) + +ALTER ROLE CURRENT_USER RESET application_name; +ALTER ROLE SESSION_USER RESET application_name; +ALTER ROLE "current_user" RESET application_name; +ALTER ROLE "Public" RESET application_name; +ALTER ROLE ALL RESET application_name; +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +----+------+------------+----------- +(0 rows) + +ALTER ROLE CURRENT_ROLE SET application_name to 'BAZ'; -- error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ALTER ROLE CURRENT_ROLE SET application_name to 'BAZ'; + ^ +ALTER ROLE USER SET application_name to 'BOOM'; -- error +ERROR: syntax error at or near "USER" +LINE 1: ALTER ROLE USER SET application_name to 'BOOM'; + ^ +ALTER ROLE PUBLIC SET application_name to 'BOMB'; -- error +ERROR: role "public" does not exist +ALTER ROLE nonexistent SET application_name to 'BOMB'; -- error +ERROR: role "nonexistent" does not exist +-- ALTER USER SET/RESET +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +----+------+------------+----------- +(0 rows) + +ALTER USER CURRENT_USER SET application_name to 'FOO'; +ALTER USER SESSION_USER SET application_name to 'BAR'; +ALTER USER "current_user" SET application_name to 'FOOFOO'; +ALTER USER "Public" SET application_name to 'BARBAR'; +ALTER USER ALL SET application_name to 'SLAP'; +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +-----+------------------+--------------+--------------------------- + ALL | Public | - | {application_name=BARBAR} + ALL | current_user | - | {application_name=FOOFOO} + ALL | regress_testrol1 | session_user | {application_name=BAR} + ALL | regress_testrol2 | current_user | {application_name=FOO} +(4 rows) + +ALTER USER regress_testrol1 SET application_name to 'SLAM'; +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +-----+------------------+--------------+--------------------------- + ALL | Public | - | {application_name=BARBAR} + ALL | current_user | - | {application_name=FOOFOO} + ALL | regress_testrol1 | session_user | {application_name=SLAM} + ALL | regress_testrol2 | current_user | {application_name=FOO} +(4 rows) + +ALTER USER CURRENT_USER RESET application_name; +ALTER USER SESSION_USER RESET application_name; +ALTER USER "current_user" RESET application_name; +ALTER USER "Public" RESET application_name; +ALTER USER ALL RESET application_name; +SELECT * FROM chksetconfig(); + db | role | rolkeyword | setconfig +----+------+------------+----------- +(0 rows) + +ALTER USER CURRENT_USER SET application_name to 'BAZ'; -- error +ALTER USER USER SET application_name to 'BOOM'; -- error +ERROR: syntax error at or near "USER" +LINE 1: ALTER USER USER SET application_name to 'BOOM'; + ^ +ALTER USER PUBLIC SET application_name to 'BOMB'; -- error +ERROR: role "public" does not exist +ALTER USER NONE SET application_name to 'BOMB'; -- error +ERROR: role name "none" is reserved +LINE 1: ALTER USER NONE SET application_name to 'BOMB'; + ^ +ALTER USER nonexistent SET application_name to 'BOMB'; -- error +ERROR: role "nonexistent" does not exist +-- CREATE SCHEMA +set client_min_messages to error; +CREATE SCHEMA newschema1 AUTHORIZATION CURRENT_USER; +CREATE SCHEMA newschema2 AUTHORIZATION "current_user"; +CREATE SCHEMA newschema3 AUTHORIZATION SESSION_USER; +CREATE SCHEMA newschema4 AUTHORIZATION regress_testrolx; +CREATE SCHEMA newschema5 AUTHORIZATION "Public"; +CREATE SCHEMA newschema6 AUTHORIZATION USER; -- error +ERROR: syntax error at or near "USER" +LINE 1: CREATE SCHEMA newschema6 AUTHORIZATION USER; + ^ +CREATE SCHEMA newschema6 AUTHORIZATION CURRENT_ROLE; -- error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: CREATE SCHEMA newschema6 AUTHORIZATION CURRENT_ROLE; + ^ +CREATE SCHEMA newschema6 AUTHORIZATION PUBLIC; -- error +ERROR: role "public" does not exist +CREATE SCHEMA newschema6 AUTHORIZATION "public"; -- error +ERROR: role "public" does not exist +CREATE SCHEMA newschema6 AUTHORIZATION NONE; -- error +ERROR: role name "none" is reserved +LINE 1: CREATE SCHEMA newschema6 AUTHORIZATION NONE; + ^ +CREATE SCHEMA newschema6 AUTHORIZATION nonexistent; -- error +ERROR: role "nonexistent" does not exist +SELECT n.nspname, r.rolname FROM pg_namespace n + JOIN pg_roles r ON (r.oid = n.nspowner) + WHERE n.nspname LIKE 'newschema_' ORDER BY 1; + nspname | rolname +------------+------------------ + newschema1 | regress_testrol2 + newschema2 | current_user + newschema3 | regress_testrol1 + newschema4 | regress_testrolx + newschema5 | Public +(5 rows) + +CREATE SCHEMA IF NOT EXISTS newschema1 AUTHORIZATION CURRENT_USER; +CREATE SCHEMA IF NOT EXISTS newschema2 AUTHORIZATION "current_user"; +CREATE SCHEMA IF NOT EXISTS newschema3 AUTHORIZATION SESSION_USER; +CREATE SCHEMA IF NOT EXISTS newschema4 AUTHORIZATION regress_testrolx; +CREATE SCHEMA IF NOT EXISTS newschema5 AUTHORIZATION "Public"; +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION USER; -- error +ERROR: syntax error at or near "USER" +LINE 1: CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION USER; + ^ +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION CURRENT_ROLE; -- error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ...ATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION CURRENT_RO... + ^ +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION PUBLIC; -- error +ERROR: role "public" does not exist +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION "public"; -- error +ERROR: role "public" does not exist +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION NONE; -- error +ERROR: role name "none" is reserved +LINE 1: CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION NONE; + ^ +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION nonexistent; -- error +ERROR: role "nonexistent" does not exist +SELECT n.nspname, r.rolname FROM pg_namespace n + JOIN pg_roles r ON (r.oid = n.nspowner) + WHERE n.nspname LIKE 'newschema_' ORDER BY 1; + nspname | rolname +------------+------------------ + newschema1 | regress_testrol2 + newschema2 | current_user + newschema3 | regress_testrol1 + newschema4 | regress_testrolx + newschema5 | Public +(5 rows) + +-- ALTER TABLE OWNER TO +\c - +SET SESSION AUTHORIZATION regress_testrol0; +set client_min_messages to error; +CREATE TABLE testtab1 (a int); +CREATE TABLE testtab2 (a int); +CREATE TABLE testtab3 (a int); +CREATE TABLE testtab4 (a int); +CREATE TABLE testtab5 (a int); +CREATE TABLE testtab6 (a int); +\c - +SET SESSION AUTHORIZATION regress_testrol1; +SET ROLE regress_testrol2; +ALTER TABLE testtab1 OWNER TO CURRENT_USER; +ALTER TABLE testtab2 OWNER TO "current_user"; +ALTER TABLE testtab3 OWNER TO SESSION_USER; +ALTER TABLE testtab4 OWNER TO regress_testrolx; +ALTER TABLE testtab5 OWNER TO "Public"; +ALTER TABLE testtab6 OWNER TO CURRENT_ROLE; -- error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ALTER TABLE testtab6 OWNER TO CURRENT_ROLE; + ^ +ALTER TABLE testtab6 OWNER TO USER; --error +ERROR: syntax error at or near "USER" +LINE 1: ALTER TABLE testtab6 OWNER TO USER; + ^ +ALTER TABLE testtab6 OWNER TO PUBLIC; -- error +ERROR: role "public" does not exist +ALTER TABLE testtab6 OWNER TO "public"; -- error +ERROR: role "public" does not exist +ALTER TABLE testtab6 OWNER TO nonexistent; -- error +ERROR: role "nonexistent" does not exist +SELECT c.relname, r.rolname + FROM pg_class c JOIN pg_roles r ON (r.oid = c.relowner) + WHERE relname LIKE 'testtab_' + ORDER BY 1; + relname | rolname +----------+------------------ + testtab1 | regress_testrol2 + testtab2 | current_user + testtab3 | regress_testrol1 + testtab4 | regress_testrolx + testtab5 | Public + testtab6 | regress_testrol0 +(6 rows) + +-- ALTER TABLE, VIEW, MATERIALIZED VIEW, FOREIGN TABLE, SEQUENCE are +-- changed their owner in the same way. +-- ALTER FUNCTION +\c - +SET SESSION AUTHORIZATION regress_testrol0; +CREATE FUNCTION testagg1(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg2(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg3(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg4(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg5(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg5(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +ERROR: function "testagg5" already exists with same argument types +CREATE FUNCTION testagg6(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg7(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg8(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg9(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +\c - +SET SESSION AUTHORIZATION regress_testrol1; +SET ROLE regress_testrol2; +ALTER FUNCTION testagg1(int2) OWNER TO CURRENT_USER; +ALTER FUNCTION testagg2(int2) OWNER TO "current_user"; +ALTER FUNCTION testagg3(int2) OWNER TO SESSION_USER; +ALTER FUNCTION testagg4(int2) OWNER TO regress_testrolx; +ALTER FUNCTION testagg5(int2) OWNER TO "Public"; +ALTER FUNCTION testagg5(int2) OWNER TO CURRENT_ROLE; -- error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ALTER FUNCTION testagg5(int2) OWNER TO CURRENT_ROLE; + ^ +ALTER FUNCTION testagg5(int2) OWNER TO USER; -- error +ERROR: syntax error at or near "USER" +LINE 1: ALTER FUNCTION testagg5(int2) OWNER TO USER; + ^ +ALTER FUNCTION testagg5(int2) OWNER TO PUBLIC; -- error +ERROR: role "public" does not exist +ALTER FUNCTION testagg5(int2) OWNER TO "public"; -- error +ERROR: role "public" does not exist +ALTER FUNCTION testagg5(int2) OWNER TO nonexistent; -- error +ERROR: role "nonexistent" does not exist +SELECT p.proname, r.rolname + FROM pg_proc p JOIN pg_roles r ON (r.oid = p.proowner) + WHERE proname LIKE 'testagg_' + ORDER BY 1; + proname | rolname +----------+------------------ + testagg1 | regress_testrol2 + testagg2 | current_user + testagg3 | regress_testrol1 + testagg4 | regress_testrolx + testagg5 | Public + testagg6 | regress_testrol0 + testagg7 | regress_testrol0 + testagg8 | regress_testrol0 + testagg9 | regress_testrol0 +(9 rows) + +-- NOT SUPPORTED +-- +-- -- CREATE USER MAPPING +-- CREATE FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv1 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv2 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv3 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv4 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv5 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv6 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv7 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv8 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv9 FOREIGN DATA WRAPPER test_wrapper; +-- +-- -- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE USER MAPPING FOR CURRENT_USER SERVER sv1 OPTIONS (user 'CURRENT_USER'); +ERROR: Generic OPTIONS not supported yet +LINE 1: CREATE USER MAPPING FOR CURRENT_USER SERVER sv1 OPTIONS (use... + ^ +HINT: Please report the issue on https://github.com/YugaByte/yugabyte-db/issues +-- CREATE USER MAPPING FOR "current_user" SERVER sv2 OPTIONS (user '"current_user"'); +-- CREATE USER MAPPING FOR USER SERVER sv3 OPTIONS (user 'USER'); +-- CREATE USER MAPPING FOR "user" SERVER sv4 OPTIONS (user '"USER"'); +-- CREATE USER MAPPING FOR SESSION_USER SERVER sv5 OPTIONS (user 'SESSION_USER'); +-- CREATE USER MAPPING FOR PUBLIC SERVER sv6 OPTIONS (user 'PUBLIC'); +-- CREATE USER MAPPING FOR "Public" SERVER sv7 OPTIONS (user '"Public"'); +-- CREATE USER MAPPING FOR regress_testrolx SERVER sv8 OPTIONS (user 'regress_testrolx'); +-- +-- CREATE USER MAPPING FOR CURRENT_ROLE SERVER sv9 +-- OPTIONS (user 'CURRENT_ROLE'); -- error +-- CREATE USER MAPPING FOR nonexistent SERVER sv9 +-- OPTIONS (user 'nonexistent'); -- error; +-- +-- SELECT * FROM chkumapping(); +-- +-- -- ALTER USER MAPPING +-- ALTER USER MAPPING FOR CURRENT_USER SERVER sv1 +-- OPTIONS (SET user 'CURRENT_USER_alt'); +-- ALTER USER MAPPING FOR "current_user" SERVER sv2 +-- OPTIONS (SET user '"current_user"_alt'); +-- ALTER USER MAPPING FOR USER SERVER sv3 +-- OPTIONS (SET user 'USER_alt'); +-- ALTER USER MAPPING FOR "user" SERVER sv4 +-- OPTIONS (SET user '"user"_alt'); +-- ALTER USER MAPPING FOR SESSION_USER SERVER sv5 +-- OPTIONS (SET user 'SESSION_USER_alt'); +-- ALTER USER MAPPING FOR PUBLIC SERVER sv6 +-- OPTIONS (SET user 'public_alt'); +-- ALTER USER MAPPING FOR "Public" SERVER sv7 +-- OPTIONS (SET user '"Public"_alt'); +-- ALTER USER MAPPING FOR regress_testrolx SERVER sv8 +-- OPTIONS (SET user 'regress_testrolx_alt'); +-- +-- ALTER USER MAPPING FOR CURRENT_ROLE SERVER sv9 +-- OPTIONS (SET user 'CURRENT_ROLE_alt'); +-- ALTER USER MAPPING FOR nonexistent SERVER sv9 +-- OPTIONS (SET user 'nonexistent_alt'); -- error +-- +-- SELECT * FROM chkumapping(); +-- +-- -- DROP USER MAPPING +-- DROP USER MAPPING FOR CURRENT_USER SERVER sv1; +-- DROP USER MAPPING FOR "current_user" SERVER sv2; +-- DROP USER MAPPING FOR USER SERVER sv3; +-- DROP USER MAPPING FOR "user" SERVER sv4; +-- DROP USER MAPPING FOR SESSION_USER SERVER sv5; +-- DROP USER MAPPING FOR PUBLIC SERVER sv6; +-- DROP USER MAPPING FOR "Public" SERVER sv7; +-- DROP USER MAPPING FOR regress_testrolx SERVER sv8; +-- +-- DROP USER MAPPING FOR CURRENT_ROLE SERVER sv9; -- error +-- DROP USER MAPPING FOR nonexistent SERVER sv; -- error +-- SELECT * FROM chkumapping(); +-- +-- CREATE USER MAPPING FOR CURRENT_USER SERVER sv1 OPTIONS (user 'CURRENT_USER'); +-- CREATE USER MAPPING FOR "current_user" SERVER sv2 OPTIONS (user '"current_user"'); +-- CREATE USER MAPPING FOR USER SERVER sv3 OPTIONS (user 'USER'); +-- CREATE USER MAPPING FOR "user" SERVER sv4 OPTIONS (user '"USER"'); +-- CREATE USER MAPPING FOR SESSION_USER SERVER sv5 OPTIONS (user 'SESSION_USER'); +-- CREATE USER MAPPING FOR PUBLIC SERVER sv6 OPTIONS (user 'PUBLIC'); +-- CREATE USER MAPPING FOR "Public" SERVER sv7 OPTIONS (user '"Public"'); +-- CREATE USER MAPPING FOR regress_testrolx SERVER sv8 OPTIONS (user 'regress_testrolx'); +-- SELECT * FROM chkumapping(); +-- +-- -- DROP USER MAPPING IF EXISTS +-- DROP USER MAPPING IF EXISTS FOR CURRENT_USER SERVER sv1; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR "current_user" SERVER sv2; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR USER SERVER sv3; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR "user" SERVER sv4; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR SESSION_USER SERVER sv5; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR PUBLIC SERVER sv6; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR "Public" SERVER sv7; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR regress_testrolx SERVER sv8; +-- SELECT * FROM chkumapping(); +-- +-- DROP USER MAPPING IF EXISTS FOR CURRENT_ROLE SERVER sv9; --error +-- DROP USER MAPPING IF EXISTS FOR nonexistent SERVER sv9; -- error +-- +-- GRANT/REVOKE +GRANT regress_testrol0 TO pg_signal_backend; -- success +SET ROLE pg_signal_backend; --success +RESET ROLE; +CREATE SCHEMA test_roles_schema AUTHORIZATION pg_signal_backend; --success +SET ROLE regress_testrol2; +UPDATE pg_proc SET proacl = null WHERE proname LIKE 'testagg_'; +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + proname | proacl +----------+-------- + testagg1 | + testagg2 | + testagg3 | + testagg4 | + testagg5 | + testagg6 | + testagg7 | + testagg8 | + testagg9 | +(9 rows) + +REVOKE ALL PRIVILEGES ON FUNCTION testagg1(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg2(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg3(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg4(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg5(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg6(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg7(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg8(int2) FROM PUBLIC; +GRANT ALL PRIVILEGES ON FUNCTION testagg1(int2) TO PUBLIC; +GRANT ALL PRIVILEGES ON FUNCTION testagg2(int2) TO CURRENT_USER; +GRANT ALL PRIVILEGES ON FUNCTION testagg3(int2) TO "current_user"; +GRANT ALL PRIVILEGES ON FUNCTION testagg4(int2) TO SESSION_USER; +GRANT ALL PRIVILEGES ON FUNCTION testagg5(int2) TO "Public"; +GRANT ALL PRIVILEGES ON FUNCTION testagg6(int2) TO regress_testrolx; +GRANT ALL PRIVILEGES ON FUNCTION testagg7(int2) TO "public"; +GRANT ALL PRIVILEGES ON FUNCTION testagg8(int2) + TO current_user, public, regress_testrolx; +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + proname | proacl +----------+----------------------------------------------------------------------------------------------------------------------------------- + testagg1 | {regress_testrol2=X/regress_testrol2,=X/regress_testrol2} + testagg2 | {current_user=X/current_user,regress_testrol2=X/current_user} + testagg3 | {regress_testrol1=X/regress_testrol1,current_user=X/regress_testrol1} + testagg4 | {regress_testrolx=X/regress_testrolx,regress_testrol1=X/regress_testrolx} + testagg5 | {Public=X/Public} + testagg6 | {regress_testrol0=X/regress_testrol0,regress_testrolx=X/regress_testrol0} + testagg7 | {regress_testrol0=X/regress_testrol0,=X/regress_testrol0} + testagg8 | {regress_testrol0=X/regress_testrol0,regress_testrol2=X/regress_testrol0,=X/regress_testrol0,regress_testrolx=X/regress_testrol0} + testagg9 | +(9 rows) + +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO CURRENT_ROLE; --error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ...RANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO CURRENT_RO... + ^ +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO USER; --error +ERROR: syntax error at or near "USER" +LINE 1: GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO USER; + ^ +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO NONE; --error +ERROR: role name "none" is reserved +LINE 1: GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO NONE; + ^ +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO "none"; --error +ERROR: role name "none" is reserved +LINE 1: GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO "none"; + ^ +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + proname | proacl +----------+----------------------------------------------------------------------------------------------------------------------------------- + testagg1 | {regress_testrol2=X/regress_testrol2,=X/regress_testrol2} + testagg2 | {current_user=X/current_user,regress_testrol2=X/current_user} + testagg3 | {regress_testrol1=X/regress_testrol1,current_user=X/regress_testrol1} + testagg4 | {regress_testrolx=X/regress_testrolx,regress_testrol1=X/regress_testrolx} + testagg5 | {Public=X/Public} + testagg6 | {regress_testrol0=X/regress_testrol0,regress_testrolx=X/regress_testrol0} + testagg7 | {regress_testrol0=X/regress_testrol0,=X/regress_testrol0} + testagg8 | {regress_testrol0=X/regress_testrol0,regress_testrol2=X/regress_testrol0,=X/regress_testrol0,regress_testrolx=X/regress_testrol0} + testagg9 | +(9 rows) + +REVOKE ALL PRIVILEGES ON FUNCTION testagg1(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg2(int2) FROM CURRENT_USER; +REVOKE ALL PRIVILEGES ON FUNCTION testagg3(int2) FROM "current_user"; +REVOKE ALL PRIVILEGES ON FUNCTION testagg4(int2) FROM SESSION_USER; +REVOKE ALL PRIVILEGES ON FUNCTION testagg5(int2) FROM "Public"; +REVOKE ALL PRIVILEGES ON FUNCTION testagg6(int2) FROM regress_testrolx; +REVOKE ALL PRIVILEGES ON FUNCTION testagg7(int2) FROM "public"; +REVOKE ALL PRIVILEGES ON FUNCTION testagg8(int2) + FROM current_user, public, regress_testrolx; +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + proname | proacl +----------+--------------------------------------- + testagg1 | {regress_testrol2=X/regress_testrol2} + testagg2 | {current_user=X/current_user} + testagg3 | {regress_testrol1=X/regress_testrol1} + testagg4 | {regress_testrolx=X/regress_testrolx} + testagg5 | {} + testagg6 | {regress_testrol0=X/regress_testrol0} + testagg7 | {regress_testrol0=X/regress_testrol0} + testagg8 | {regress_testrol0=X/regress_testrol0} + testagg9 | +(9 rows) + +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM CURRENT_ROLE; --error +ERROR: syntax error at or near "CURRENT_ROLE" +LINE 1: ...KE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM CURRENT_RO... + ^ +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM USER; --error +ERROR: syntax error at or near "USER" +LINE 1: REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM USER; + ^ +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM NONE; --error +ERROR: role name "none" is reserved +LINE 1: REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM NONE; + ^ +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM "none"; --error +ERROR: role name "none" is reserved +LINE 1: ...EVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM "none"; + ^ +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + proname | proacl +----------+--------------------------------------- + testagg1 | {regress_testrol2=X/regress_testrol2} + testagg2 | {current_user=X/current_user} + testagg3 | {regress_testrol1=X/regress_testrol1} + testagg4 | {regress_testrolx=X/regress_testrolx} + testagg5 | {} + testagg6 | {regress_testrol0=X/regress_testrol0} + testagg7 | {regress_testrol0=X/regress_testrol0} + testagg8 | {regress_testrol0=X/regress_testrol0} + testagg9 | +(9 rows) + +-- clean up +\c +DROP SCHEMA test_roles_schema; +DROP OWNED BY regress_testrol0, "Public", "current_user", regress_testrol1, regress_testrol2, regress_testrolx CASCADE; +DROP ROLE regress_testrol0, regress_testrol1, regress_testrol2, regress_testrolx; +DROP ROLE "Public", "None", "current_user", "session_user", "user"; diff --git a/src/postgres/src/test/regress/sql/yb_init_privs.sql b/src/postgres/src/test/regress/sql/yb_init_privs.sql new file mode 100644 index 000000000000..4a31af27986a --- /dev/null +++ b/src/postgres/src/test/regress/sql/yb_init_privs.sql @@ -0,0 +1,10 @@ +-- Test initial privileges + +-- There should always be some initial privileges, set up by initdb +SELECT count(*) > 0 FROM pg_init_privs; + +-- Intentionally include some non-initial privs for pg_dump to dump out +GRANT SELECT ON pg_proc TO CURRENT_USER; +GRANT SELECT (prosrc) ON pg_proc TO CURRENT_USER; + +GRANT SELECT (rolname, rolsuper) ON pg_authid TO CURRENT_USER; diff --git a/src/postgres/src/test/regress/sql/yb_password.sql b/src/postgres/src/test/regress/sql/yb_password.sql new file mode 100644 index 000000000000..8f8252d127f0 --- /dev/null +++ b/src/postgres/src/test/regress/sql/yb_password.sql @@ -0,0 +1,79 @@ +-- +-- Tests for password verifiers +-- + +-- Tests for GUC password_encryption +SET password_encryption = 'novalue'; -- error +SET password_encryption = true; -- ok +SET password_encryption = 'md5'; -- ok +SET password_encryption = 'scram-sha-256'; -- ok + +-- consistency of password entries +SET password_encryption = 'md5'; +CREATE ROLE regress_passwd1 PASSWORD 'role_pwd1'; +SET password_encryption = 'on'; +CREATE ROLE regress_passwd2 PASSWORD 'role_pwd2'; +SET password_encryption = 'scram-sha-256'; +CREATE ROLE regress_passwd3 PASSWORD 'role_pwd3'; +CREATE ROLE regress_passwd4 PASSWORD NULL; + +-- check list of created entries +-- +-- The scram verifier will look something like: +-- SCRAM-SHA-256$4096:E4HxLGtnRzsYwg==$6YtlR4t69SguDiwFvbVgVZtuz6gpJQQqUMZ7IQJK5yI=:ps75jrHeYU4lXCcXI4O8oIdJ3eO8o2jirjruw9phBTo= +-- +-- Since the salt is random, the exact value stored will be different on every test +-- run. Use a regular expression to mask the changing parts. +SELECT rolname, regexp_replace(rolpassword, '(SCRAM-SHA-256)\$(\d+):([a-zA-Z0-9+/=]+)\$([a-zA-Z0-9+=/]+):([a-zA-Z0-9+/=]+)', '\1$\2:$:') as rolpassword_masked + FROM pg_authid + WHERE rolname LIKE 'regress_passwd%' + ORDER BY rolname, rolpassword; + +-- Rename a role +ALTER ROLE regress_passwd2 RENAME TO regress_passwd2_new; +-- md5 entry should have been removed +SELECT rolname, rolpassword + FROM pg_authid + WHERE rolname LIKE 'regress_passwd2_new' + ORDER BY rolname, rolpassword; +ALTER ROLE regress_passwd2_new RENAME TO regress_passwd2; + +-- Change passwords with ALTER USER. With plaintext or already-encrypted +-- passwords. +SET password_encryption = 'md5'; + +-- encrypt with MD5 +ALTER ROLE regress_passwd2 PASSWORD 'foo'; +-- already encrypted, use as they are +ALTER ROLE regress_passwd1 PASSWORD 'md5cd3578025fe2c3d7ed1b9a9b26238b70'; +ALTER ROLE regress_passwd3 PASSWORD 'SCRAM-SHA-256$4096:VLK4RMaQLCvNtQ==$6YtlR4t69SguDiwFvbVgVZtuz6gpJQQqUMZ7IQJK5yI=:ps75jrHeYU4lXCcXI4O8oIdJ3eO8o2jirjruw9phBTo='; + +SET password_encryption = 'scram-sha-256'; +-- create SCRAM verifier +ALTER ROLE regress_passwd4 PASSWORD 'foo'; +-- already encrypted with MD5, use as it is +CREATE ROLE regress_passwd5 PASSWORD 'md5e73a4b11df52a6068f8b39f90be36023'; + +SELECT rolname, regexp_replace(rolpassword, '(SCRAM-SHA-256)\$(\d+):([a-zA-Z0-9+/=]+)\$([a-zA-Z0-9+=/]+):([a-zA-Z0-9+/=]+)', '\1$\2:$:') as rolpassword_masked + FROM pg_authid + WHERE rolname LIKE 'regress_passwd%' + ORDER BY rolname, rolpassword; + +-- An empty password is not allowed, in any form +CREATE ROLE regress_passwd_empty PASSWORD ''; +ALTER ROLE regress_passwd_empty PASSWORD 'md585939a5ce845f1a1b620742e3c659e0a'; +ALTER ROLE regress_passwd_empty PASSWORD 'SCRAM-SHA-256$4096:hpFyHTUsSWcR7O9P$LgZFIt6Oqdo27ZFKbZ2nV+vtnYM995pDh9ca6WSi120=:qVV5NeluNfUPkwm7Vqat25RjSPLkGeoZBQs6wVv+um4='; +SELECT rolpassword FROM pg_authid WHERE rolname='regress_passwd_empty'; + +DROP ROLE regress_passwd1; +DROP ROLE regress_passwd2; +DROP ROLE regress_passwd3; +DROP ROLE regress_passwd4; +DROP ROLE regress_passwd5; +DROP ROLE regress_passwd_empty; + +-- all entries should have been removed +SELECT rolname, rolpassword + FROM pg_authid + WHERE rolname LIKE 'regress_passwd%' + ORDER BY rolname, rolpassword; diff --git a/src/postgres/src/test/regress/sql/yb_privileges.sql b/src/postgres/src/test/regress/sql/yb_privileges.sql new file mode 100644 index 000000000000..5160e9327c15 --- /dev/null +++ b/src/postgres/src/test/regress/sql/yb_privileges.sql @@ -0,0 +1,1282 @@ +-- +-- Test access privileges +-- + +-- Clean up in case a prior regression run failed + +-- Suppress NOTICE messages when users/groups don't exist +SET client_min_messages TO 'warning'; + +DROP ROLE IF EXISTS regress_priv_group1; +DROP ROLE IF EXISTS regress_priv_group2; + +DROP ROLE IF EXISTS regress_priv_user1; +DROP ROLE IF EXISTS regress_priv_user2; +DROP ROLE IF EXISTS regress_priv_user3; +DROP ROLE IF EXISTS regress_priv_user4; +DROP ROLE IF EXISTS regress_priv_user5; +DROP ROLE IF EXISTS regress_priv_user6; + +SELECT lo_unlink(oid) FROM pg_largeobject_metadata WHERE oid >= 1000 AND oid < 3000 ORDER BY oid; + +RESET client_min_messages; + +-- test proper begins here + +CREATE USER regress_priv_user1; +CREATE USER regress_priv_user2; +CREATE USER regress_priv_user3; +CREATE USER regress_priv_user4; +CREATE USER regress_priv_user5; +CREATE USER regress_priv_user5; -- duplicate + +CREATE GROUP regress_priv_group1; +CREATE GROUP regress_priv_group2 WITH USER regress_priv_user1, regress_priv_user2; + +ALTER GROUP regress_priv_group1 ADD USER regress_priv_user4; + +ALTER GROUP regress_priv_group2 ADD USER regress_priv_user2; -- duplicate +ALTER GROUP regress_priv_group2 DROP USER regress_priv_user2; +GRANT regress_priv_group2 TO regress_priv_user4 WITH ADMIN OPTION; + +-- test owner privileges + +SET SESSION AUTHORIZATION regress_priv_user1; +SELECT session_user, current_user; + +CREATE TABLE atest1 ( a int, b text ); +SELECT * FROM atest1; +INSERT INTO atest1 VALUES (1, 'one'); +DELETE FROM atest1; +UPDATE atest1 SET a = 1 WHERE b = 'blech'; +TRUNCATE atest1; + +REVOKE ALL ON atest1 FROM PUBLIC; +SELECT * FROM atest1; + +GRANT ALL ON atest1 TO regress_priv_user2; +GRANT SELECT ON atest1 TO regress_priv_user3, regress_priv_user4; +SELECT * FROM atest1; + +CREATE TABLE atest2 (col1 varchar(10), col2 boolean); +GRANT SELECT ON atest2 TO regress_priv_user2; +GRANT UPDATE ON atest2 TO regress_priv_user3; +GRANT INSERT ON atest2 TO regress_priv_user4; +GRANT TRUNCATE ON atest2 TO regress_priv_user5; + +SET SESSION AUTHORIZATION regress_priv_user2; +SELECT session_user, current_user; + +-- try various combinations of queries on atest1 and atest2 + +SELECT * FROM atest1; -- ok +SELECT * FROM atest2; -- ok +INSERT INTO atest1 VALUES (2, 'two'); -- ok +INSERT INTO atest2 VALUES ('foo', true); -- fail +INSERT INTO atest1 SELECT 1, b FROM atest1; -- ok +UPDATE atest1 SET a = 1 WHERE a = 2; -- ok +UPDATE atest2 SET col2 = NOT col2; -- fail +DELETE FROM atest2; -- fail +TRUNCATE atest2; -- fail +COPY atest2 FROM stdin; -- fail +GRANT ALL ON atest1 TO PUBLIC; -- fail + +-- checks in subquery, both ok +SELECT * FROM atest1 WHERE ( b IN ( SELECT col1 FROM atest2 ) ); +SELECT * FROM atest2 WHERE ( col1 IN ( SELECT b FROM atest1 ) ); + + +SET SESSION AUTHORIZATION regress_priv_user3; +SELECT session_user, current_user; + +SELECT * FROM atest1; -- ok +SELECT * FROM atest2; -- fail +INSERT INTO atest1 VALUES (2, 'two'); -- fail +INSERT INTO atest2 VALUES ('foo', true); -- fail +INSERT INTO atest1 SELECT 1, b FROM atest1; -- fail +UPDATE atest1 SET a = 1 WHERE a = 2; -- fail +UPDATE atest2 SET col2 = NULL; -- ok +UPDATE atest2 SET col2 = NOT col2; -- fails; requires SELECT on atest2 +DELETE FROM atest2; -- fail +TRUNCATE atest2; -- fail +COPY atest2 FROM stdin; -- fail + +-- checks in subquery, both fail +SELECT * FROM atest1 WHERE ( b IN ( SELECT col1 FROM atest2 ) ); +SELECT * FROM atest2 WHERE ( col1 IN ( SELECT b FROM atest1 ) ); + +SET SESSION AUTHORIZATION regress_priv_user4; +COPY atest2 FROM stdin; -- ok +bar true +\. +SELECT * FROM atest1; -- ok + +-- NOT SUPPORTED +-- +-- -- test leaky-function protections in selfuncs +-- +-- -- regress_priv_user1 will own a table and provide a view for it. +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- +-- CREATE TABLE atest12 as +-- SELECT x AS a, 10001 - x AS b FROM generate_series(1,10000) x; +-- CREATE INDEX ON atest12 (a); +-- CREATE INDEX ON atest12 (abs(a)); +-- +-- CREATE FUNCTION leak(integer,integer) RETURNS boolean +-- AS $$begin return $1 < $2; end$$ +-- LANGUAGE plpgsql immutable; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE OPERATOR <<< (procedure = leak, leftarg = integer, rightarg = integer, + restrict = scalarltsel); +-- +-- -- view with leaky operator +-- CREATE VIEW atest12v AS +-- SELECT * FROM atest12 WHERE b <<< 5; +-- GRANT SELECT ON atest12v TO PUBLIC; +-- +-- -- This plan should use nestloop, knowing that few rows will be selected. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12v x, atest12v y WHERE x.a = y.b; +-- +-- -- And this one. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12 x, atest12 y +-- WHERE x.a = y.b and abs(y.a) <<< 5; +-- +-- -- Check if regress_priv_user2 can break security. +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- +-- CREATE FUNCTION leak2(integer,integer) RETURNS boolean +-- AS $$begin raise notice 'leak % %', $1, $2; return $1 > $2; end$$ +-- LANGUAGE plpgsql immutable; +-- CREATE OPERATOR >>> (procedure = leak2, leftarg = integer, rightarg = integer, +-- restrict = scalargtsel); +-- +-- -- This should not show any "leak" notices before failing. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12 WHERE a >>> 0; +-- +-- -- This plan should use hashjoin, as it will expect many rows to be selected. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12v x, atest12v y WHERE x.a = y.b; +-- +-- -- Now regress_priv_user1 grants sufficient access to regress_priv_user2. +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- GRANT SELECT (a, b) ON atest12 TO PUBLIC; +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- +-- -- Now regress_priv_user2 will also get a good row estimate. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12v x, atest12v y WHERE x.a = y.b; +-- +-- -- But not for this, due to lack of table-wide permissions needed +-- -- to make use of the expression index's statistics. +-- EXPLAIN (COSTS OFF) SELECT * FROM atest12 x, atest12 y +-- WHERE x.a = y.b and abs(y.a) <<< 5; +-- +-- clean up (regress_priv_user1's objects are all dropped later) +-- DROP FUNCTION leak2(integer, integer) CASCADE; +-- + +-- groups + +SET SESSION AUTHORIZATION regress_priv_user3; +CREATE TABLE atest3 (one int, two int, three int); +GRANT DELETE ON atest3 TO GROUP regress_priv_group2; + +SET SESSION AUTHORIZATION regress_priv_user1; + +SELECT * FROM atest3; -- fail +DELETE FROM atest3; -- ok + + +-- views + +SET SESSION AUTHORIZATION regress_priv_user3; + +CREATE VIEW atestv1 AS SELECT * FROM atest1; -- ok +/* The next *should* fail, but it's not implemented that way yet. */ +CREATE VIEW atestv2 AS SELECT * FROM atest2; +CREATE VIEW atestv3 AS SELECT * FROM atest3; -- ok +/* Empty view is a corner case that failed in 9.2. */ +CREATE VIEW atestv0 AS SELECT 0 as x WHERE false; -- ok + +SELECT * FROM atestv1; -- ok +SELECT * FROM atestv2; -- fail +GRANT SELECT ON atestv1, atestv3 TO regress_priv_user4; +GRANT SELECT ON atestv2 TO regress_priv_user2; + +SET SESSION AUTHORIZATION regress_priv_user4; + +SELECT * FROM atestv1; -- ok +SELECT * FROM atestv2; -- fail +SELECT * FROM atestv3; -- ok +SELECT * FROM atestv0; -- fail + +RESET SESSION AUTHORIZATION; + +CREATE TABLE priv_int8_tbl(q1 int8, q2 int8, PRIMARY KEY (q1, q2)); +INSERT INTO priv_int8_tbl VALUES(' 123 ',' 456'); +INSERT INTO priv_int8_tbl VALUES('123 ','4567890123456789'); +INSERT INTO priv_int8_tbl VALUES('4567890123456789','123'); +INSERT INTO priv_int8_tbl VALUES(+4567890123456789,'4567890123456789'); +INSERT INTO priv_int8_tbl VALUES('+4567890123456789','-4567890123456789'); + +SET SESSION AUTHORIZATION regress_priv_user4; + +-- Appendrels excluded by constraints failed to check permissions in 8.4-9.2. +select * from + ((select a.q1 as x from priv_int8_tbl a offset 0) + union all + (select b.q2 as x from priv_int8_tbl b offset 0)) ss +where false; + +set constraint_exclusion = on; +select * from + ((select a.q1 as x, random() from priv_int8_tbl a where q1 > 0) + union all + (select b.q2 as x, random() from priv_int8_tbl b where q2 > 0)) ss +where x < 0; +reset constraint_exclusion; + +CREATE VIEW atestv4 AS SELECT * FROM atestv3; -- nested view +SELECT * FROM atestv4; -- ok +GRANT SELECT ON atestv4 TO regress_priv_user2; + +SET SESSION AUTHORIZATION regress_priv_user2; + +-- Two complex cases: + +SELECT * FROM atestv3; -- fail +SELECT * FROM atestv4; -- ok (even though regress_priv_user2 cannot access underlying atestv3) + +SELECT * FROM atest2; -- ok +SELECT * FROM atestv2; -- fail (even though regress_priv_user2 can access underlying atest2) + +-- Test column level permissions + +SET SESSION AUTHORIZATION regress_priv_user1; +CREATE SEQUENCE twoseq START 1000; +CREATE SEQUENCE fourseq START 100; +CREATE TABLE atest5 ( + one int, + two int UNIQUE NOT NULL DEFAULT NEXTVAL('twoseq'), + three int, + four int UNIQUE NOT NULL DEFAULT NEXTVAL('fourseq') +); +CREATE TABLE atest6 (one int, two int, blue int); +GRANT SELECT (one), INSERT (two), UPDATE (three) ON atest5 TO regress_priv_user4; +GRANT ALL (one) ON atest5 TO regress_priv_user3; +GRANT ALL ON SEQUENCE twoseq TO regress_priv_user3, regress_priv_user4; +GRANT ALL ON SEQUENCE fourseq TO regress_priv_user3, regress_priv_user4; + +INSERT INTO atest5 VALUES (1,2,3); + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT * FROM atest5; -- fail +SELECT one FROM atest5; -- ok +COPY atest5 (one) TO stdout; -- ok +SELECT two FROM atest5; -- fail +COPY atest5 (two) TO stdout; -- fail +SELECT atest5 FROM atest5; -- fail +COPY atest5 (one,two) TO stdout; -- fail +SELECT 1 FROM atest5; -- ok +SELECT 1 FROM atest5 a JOIN atest5 b USING (one); -- ok +SELECT 1 FROM atest5 a JOIN atest5 b USING (two); -- fail +SELECT 1 FROM atest5 a NATURAL JOIN atest5 b; -- fail +SELECT (j.*) IS NULL FROM (atest5 a JOIN atest5 b USING (one)) j; -- fail +SELECT 1 FROM atest5 WHERE two = 2; -- fail +SELECT * FROM atest1, atest5; -- fail +SELECT atest1.* FROM atest1, atest5; -- ok +SELECT atest1.*,atest5.one FROM atest1, atest5; -- ok +SELECT atest1.*,atest5.one FROM atest1 JOIN atest5 ON (atest1.a = atest5.two); -- fail +SELECT atest1.*,atest5.one FROM atest1 JOIN atest5 ON (atest1.a = atest5.one); -- ok +SELECT one, two FROM atest5; -- fail + +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT SELECT (one,two) ON atest6 TO regress_priv_user4; + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT one, two FROM atest5 NATURAL JOIN atest6; -- fail still + +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT SELECT (two) ON atest5 TO regress_priv_user4; + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT one, two FROM atest5 NATURAL JOIN atest6; -- ok now + +-- test column-level privileges for INSERT and UPDATE +-- TODO enable after #1611 +-- UPDATE atest5 SET three = 10; -- ok +INSERT INTO atest5 (two) VALUES (3); -- ok +COPY atest5 FROM stdin; -- fail +COPY atest5 (two) FROM stdin; -- ok +1 +\. +INSERT INTO atest5 (three) VALUES (4); -- fail +INSERT INTO atest5 VALUES (5,5,5); -- fail +UPDATE atest5 SET one = 8; -- fail +UPDATE atest5 SET three = 5, one = 2; -- fail +-- Check that column level privs are enforced in RETURNING +-- Ok. +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = 10; +-- Error. No SELECT on column three. +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = 10 RETURNING atest5.three; +-- Ok. May SELECT on column "one": +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = 10 RETURNING atest5.one; +-- Check that column level privileges are enforced for EXCLUDED +-- Ok. we may select one +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = EXCLUDED.one; +-- Error. No select rights on three +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set three = EXCLUDED.three; +INSERT INTO atest5(two) VALUES (6) ON CONFLICT (two) DO UPDATE set one = 8; -- fails (due to UPDATE) +INSERT INTO atest5(three) VALUES (4) ON CONFLICT (two) DO UPDATE set three = 10; -- fails (due to INSERT) + +-- Check that the columns in the inference require select privileges +INSERT INTO atest5(four) VALUES (4); -- fail + +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT INSERT (four) ON atest5 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; + +INSERT INTO atest5(four) VALUES (4) ON CONFLICT (four) DO UPDATE set three = 3; -- fails (due to SELECT) +INSERT INTO atest5(four) VALUES (4) ON CONFLICT ON CONSTRAINT atest5_four_key DO UPDATE set three = 3; -- fails (due to SELECT) +INSERT INTO atest5(four) VALUES (4); -- ok + +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT SELECT (four) ON atest5 TO regress_priv_user4; +SET SESSION AUTHORIZATION regress_priv_user4; + +INSERT INTO atest5(four) VALUES (4) ON CONFLICT (four) DO UPDATE set three = 3; -- ok +INSERT INTO atest5(four) VALUES (4) ON CONFLICT ON CONSTRAINT atest5_four_key DO UPDATE set three = 3; -- ok + +SET SESSION AUTHORIZATION regress_priv_user1; +REVOKE ALL (one) ON atest5 FROM regress_priv_user4; +GRANT SELECT (one,two,blue) ON atest6 TO regress_priv_user4; + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT one FROM atest5; -- fail +UPDATE atest5 SET one = 1; -- fail +SELECT atest6 FROM atest6; -- ok +COPY atest6 TO stdout; -- ok + +-- check error reporting with column privs +SET SESSION AUTHORIZATION regress_priv_user1; +CREATE TABLE t1 (c1 int, c2 int, c3 int check (c3 < 5), primary key (c1, c2)); +GRANT SELECT (c1) ON t1 TO regress_priv_user2; +GRANT INSERT (c1, c2, c3) ON t1 TO regress_priv_user2; +GRANT UPDATE (c1, c2, c3) ON t1 TO regress_priv_user2; + +-- seed data +INSERT INTO t1 VALUES (1, 1, 1); +INSERT INTO t1 VALUES (1, 2, 1); +INSERT INTO t1 VALUES (2, 1, 2); +INSERT INTO t1 VALUES (2, 2, 2); +INSERT INTO t1 VALUES (3, 1, 3); + +SET SESSION AUTHORIZATION regress_priv_user2; +INSERT INTO t1 (c1, c2) VALUES (1, 1); -- fail, but row not shown +INSERT INTO t1 (c1, c2) VALUES (null, null); -- fail, but see columns being inserted +INSERT INTO t1 (c3) VALUES (null); -- fail, but see columns being inserted or have SELECT +INSERT INTO t1 (c1) VALUES (5); -- fail, but see columns being inserted or have SELECT +UPDATE t1 SET c3 = 10; -- fail, but see columns with SELECT rights, or being modified + +SET SESSION AUTHORIZATION regress_priv_user1; +DROP TABLE t1; + +-- test column-level privileges when involved with DELETE +SET SESSION AUTHORIZATION regress_priv_user1; +ALTER TABLE atest6 ADD COLUMN three integer; +GRANT DELETE ON atest5 TO regress_priv_user3; +GRANT SELECT (two) ON atest5 TO regress_priv_user3; +REVOKE ALL (one) ON atest5 FROM regress_priv_user3; +GRANT SELECT (one) ON atest5 TO regress_priv_user4; + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT atest6 FROM atest6; -- fail +SELECT one FROM atest5 NATURAL JOIN atest6; -- fail + +SET SESSION AUTHORIZATION regress_priv_user1; +ALTER TABLE atest6 DROP COLUMN three; + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT atest6 FROM atest6; -- ok +SELECT one FROM atest5 NATURAL JOIN atest6; -- ok + +SET SESSION AUTHORIZATION regress_priv_user1; +ALTER TABLE atest6 DROP COLUMN two; +REVOKE SELECT (one,blue) ON atest6 FROM regress_priv_user4; + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT * FROM atest6; -- fail +SELECT 1 FROM atest6; -- fail + +SET SESSION AUTHORIZATION regress_priv_user3; +DELETE FROM atest5 WHERE one = 1; -- fail +DELETE FROM atest5 WHERE two = 2; -- ok + +-- NOT SUPPORTED +-- +-- -- check inheritance cases +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE TABLE atestp1 (f1 int, f2 int) WITH OIDS; +-- CREATE TABLE atestp2 (fx int, fy int) WITH OIDS; +-- CREATE TABLE atestc (fz int) INHERITS (atestp1, atestp2); +-- GRANT SELECT(fx,fy,oid) ON atestp2 TO regress_priv_user2; +-- GRANT SELECT(fx) ON atestc TO regress_priv_user2; +-- +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- SELECT fx FROM atestp2; -- ok +-- SELECT fy FROM atestp2; -- ok +-- SELECT atestp2 FROM atestp2; -- ok +-- SELECT oid FROM atestp2; -- ok +-- SELECT fy FROM atestc; -- fail +-- +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- GRANT SELECT(fy,oid) ON atestc TO regress_priv_user2; +-- +-- SET SESSION AUTHORIZATION regress_priv_user2; +-- SELECT fx FROM atestp2; -- still ok +-- SELECT fy FROM atestp2; -- ok +-- SELECT atestp2 FROM atestp2; -- ok +-- SELECT oid FROM atestp2; -- ok +-- + +-- privileges on functions, languages + +-- switch to superuser +\c - + +REVOKE ALL PRIVILEGES ON LANGUAGE sql FROM PUBLIC; +GRANT USAGE ON LANGUAGE sql TO regress_priv_user1; -- ok +GRANT USAGE ON LANGUAGE c TO PUBLIC; -- fail + +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT USAGE ON LANGUAGE sql TO regress_priv_user2; -- fail +CREATE FUNCTION priv_testfunc1(int) RETURNS int AS 'select 2 * $1;' LANGUAGE sql; +CREATE FUNCTION priv_testfunc2(int) RETURNS int AS 'select 3 * $1;' LANGUAGE sql; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE AGGREGATE priv_testagg1(int) (sfunc = int4pl, stype = int4); +-- +CREATE PROCEDURE priv_testproc1(int) AS 'select $1;' LANGUAGE sql; + +REVOKE ALL ON FUNCTION priv_testfunc1(int), priv_testfunc2(int) FROM PUBLIC; +GRANT EXECUTE ON FUNCTION priv_testfunc1(int), priv_testfunc2(int) TO regress_priv_user2; +REVOKE ALL ON FUNCTION priv_testproc1(int) FROM PUBLIC; -- fail, not a function +REVOKE ALL ON PROCEDURE priv_testproc1(int) FROM PUBLIC; +GRANT EXECUTE ON PROCEDURE priv_testproc1(int) TO regress_priv_user2; +GRANT USAGE ON FUNCTION priv_testfunc1(int) TO regress_priv_user3; -- semantic error +-- GRANT USAGE ON FUNCTION priv_testagg1(int) TO regress_priv_user3; -- semantic error +GRANT USAGE ON PROCEDURE priv_testproc1(int) TO regress_priv_user3; -- semantic error +GRANT ALL PRIVILEGES ON FUNCTION priv_testfunc1(int) TO regress_priv_user4; +GRANT ALL PRIVILEGES ON FUNCTION priv_testfunc_nosuch(int) TO regress_priv_user4; +-- GRANT ALL PRIVILEGES ON FUNCTION priv_testagg1(int) TO regress_priv_user4; +GRANT ALL PRIVILEGES ON PROCEDURE priv_testproc1(int) TO regress_priv_user4; + +CREATE FUNCTION priv_testfunc4(boolean) RETURNS text + AS 'select col1 from atest2 where col2 = $1;' + LANGUAGE sql SECURITY DEFINER; +GRANT EXECUTE ON FUNCTION priv_testfunc4(boolean) TO regress_priv_user3; + +SET SESSION AUTHORIZATION regress_priv_user2; +SELECT priv_testfunc1(5), priv_testfunc2(5); -- ok +CREATE FUNCTION priv_testfunc3(int) RETURNS int AS 'select 2 * $1;' LANGUAGE sql; -- fail +-- SELECT priv_testagg1(x) FROM (VALUES (1), (2), (3)) _(x); -- ok +CALL priv_testproc1(6); -- ok + +SET SESSION AUTHORIZATION regress_priv_user3; +SELECT priv_testfunc1(5); -- fail +-- SELECT priv_testagg1(x) FROM (VALUES (1), (2), (3)) _(x); -- fail +CALL priv_testproc1(6); -- fail +SELECT col1 FROM atest2 WHERE col2 = true; -- fail +SELECT priv_testfunc4(true); -- ok + +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT priv_testfunc1(5); -- ok +-- SELECT priv_testagg1(x) FROM (VALUES (1), (2), (3)) _(x); -- ok +CALL priv_testproc1(6); -- ok + +DROP FUNCTION priv_testfunc1(int); -- fail +-- DROP AGGREGATE priv_testagg1(int); -- fail +DROP PROCEDURE priv_testproc1(int); -- fail + +\c - + +DROP FUNCTION priv_testfunc1(int); -- ok +-- restore to sanity +GRANT ALL PRIVILEGES ON LANGUAGE sql TO PUBLIC; + +-- verify privilege checks on array-element coercions +SELECT '{1}'::int4[]::int8[]; +REVOKE ALL ON FUNCTION int8(integer) FROM PUBLIC; +SELECT '{1}'::int4[]::int8[]; --superuser, suceed +SET SESSION AUTHORIZATION regress_priv_user4; +SELECT '{1}'::int4[]::int8[]; --other user, fail +RESET SESSION AUTHORIZATION; +GRANT ALL ON FUNCTION int8(integer) TO PUBLIC; +SET SESSION AUTHORIZATION regress_priv_user4; + +-- NOT SUPPORTED +-- +-- -- privileges on types +-- +-- -- switch to superuser +-- \c - +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE TYPE priv_testtype1 AS (a int, b text); +-- REVOKE USAGE ON TYPE priv_testtype1 FROM PUBLIC; +-- GRANT USAGE ON TYPE priv_testtype1 TO regress_priv_user2; +-- GRANT USAGE ON TYPE _priv_testtype1 TO regress_priv_user2; -- fail +-- GRANT USAGE ON DOMAIN priv_testtype1 TO regress_priv_user2; -- fail +-- +-- CREATE DOMAIN priv_testdomain1 AS int; +-- REVOKE USAGE on DOMAIN priv_testdomain1 FROM PUBLIC; +-- GRANT USAGE ON DOMAIN priv_testdomain1 TO regress_priv_user2; +-- GRANT USAGE ON TYPE priv_testdomain1 TO regress_priv_user2; -- ok +-- +-- SET SESSION AUTHORIZATION regress_priv_user1; +-- +-- -- commands that should fail +-- +-- CREATE DOMAIN priv_testdomain2a AS priv_testdomain1; +-- +-- +-- CREATE DOMAIN priv_testdomain3a AS int; +-- CREATE FUNCTION castfunc(int) RETURNS priv_testdomain3a AS $$ SELECT $1::priv_testdomain3a $$ LANGUAGE SQL; +-- CREATE CAST (priv_testdomain1 AS priv_testdomain3a) WITH FUNCTION castfunc(int); +-- DROP FUNCTION castfunc(int) CASCADE; +-- DROP DOMAIN priv_testdomain3a; +-- +-- CREATE FUNCTION priv_testfunc5a(a priv_testdomain1) RETURNS int LANGUAGE SQL AS $$ SELECT $1 $$; +-- CREATE FUNCTION priv_testfunc6a(b int) RETURNS priv_testdomain1 LANGUAGE SQL AS $$ SELECT $1::priv_testdomain1 $$; +-- +-- CREATE OPERATOR !+! (PROCEDURE = int4pl, LEFTARG = priv_testdomain1, RIGHTARG = priv_testdomain1); +-- +-- CREATE TABLE test5a (a int, b priv_testdomain1); +-- CREATE TABLE test6a OF priv_testtype1; +-- CREATE TABLE test10a (a int[], b priv_testtype1[]); +-- +-- CREATE TABLE test9a (a int, b int); +-- ALTER TABLE test9a ADD COLUMN c priv_testdomain1; +-- ALTER TABLE test9a ALTER COLUMN b TYPE priv_testdomain1; +-- +-- CREATE TYPE test7a AS (a int, b priv_testdomain1); +-- +-- CREATE TYPE test8a AS (a int, b int); +-- ALTER TYPE test8a ADD ATTRIBUTE c priv_testdomain1; +-- ALTER TYPE test8a ALTER ATTRIBUTE b TYPE priv_testdomain1; +-- +-- CREATE TABLE test11a AS (SELECT 1::priv_testdomain1 AS a); +-- +-- REVOKE ALL ON TYPE priv_testtype1 FROM PUBLIC; +-- + +CREATE DOMAIN priv_testdomain1 AS int; + +SET SESSION AUTHORIZATION regress_priv_user2; + +-- commands that should succeed + +CREATE DOMAIN priv_testdomain2b AS priv_testdomain1; + +CREATE DOMAIN priv_testdomain3b AS int; + +-- NOT SUPPORTED +-- +-- CREATE FUNCTION castfunc(int) RETURNS priv_testdomain3b AS $$ SELECT $1::priv_testdomain3b $$ LANGUAGE SQL; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE CAST (priv_testdomain1 AS priv_testdomain3b) WITH FUNCTION castfunc(int); +-- +-- CREATE FUNCTION priv_testfunc5b(a priv_testdomain1) RETURNS int LANGUAGE SQL AS $$ SELECT $1 $$; +-- CREATE FUNCTION priv_testfunc6b(b int) RETURNS priv_testdomain1 LANGUAGE SQL AS $$ SELECT $1::priv_testdomain1 $$; +-- +-- CREATE OPERATOR !! (PROCEDURE = priv_testfunc5b, RIGHTARG = priv_testdomain1); +-- + +CREATE TABLE test5b (a int, b priv_testdomain1); + +CREATE TABLE test9b (a int, b int); +ALTER TABLE test9b ADD COLUMN c priv_testdomain1; + +-- NOT SUPPORTED +-- +-- ALTER TABLE test9b ALTER COLUMN b TYPE priv_testdomain1; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE TYPE test7b AS (a int, b priv_testdomain1); +-- +-- CREATE TYPE test8b AS (a int, b int); +-- ALTER TYPE test8b ADD ATTRIBUTE c priv_testdomain1; +-- ALTER TYPE test8b ALTER ATTRIBUTE b TYPE priv_testdomain1; +-- + +CREATE TABLE test11b AS (SELECT 1::priv_testdomain1 AS a); + +\c - +DROP DOMAIN priv_testdomain2b; + +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +DROP OPERATOR !! (NONE, priv_testdomain1); +-- DROP FUNCTION priv_testfunc5b(a priv_testdomain1); +-- DROP FUNCTION priv_testfunc6b(b int); +-- + +DROP TABLE test5b; +DROP TABLE test9b; + +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +DROP TYPE test7b; +-- DROP TYPE test8b; +-- DROP CAST (priv_testdomain1 AS priv_testdomain3b); +-- DROP FUNCTION castfunc(int) CASCADE; +-- + +DROP DOMAIN priv_testdomain3b; +DROP TABLE test11b; + +DROP DOMAIN priv_testdomain1; -- ok + + +-- truncate +SET SESSION AUTHORIZATION regress_priv_user5; +TRUNCATE atest2; -- ok +TRUNCATE atest3; -- fail + +-- has_table_privilege function + +-- bad-input checks +select has_table_privilege(NULL,'pg_authid','select'); +select has_table_privilege('pg_shad','select'); +select has_table_privilege('nosuchuser','pg_authid','select'); +select has_table_privilege('pg_authid','sel'); +select has_table_privilege(-999999,'pg_authid','update'); +select has_table_privilege(1,'select'); + +-- superuser +\c - + +select has_table_privilege(current_user,'pg_authid','select'); +select has_table_privilege(current_user,'pg_authid','insert'); + +select has_table_privilege(t2.oid,'pg_authid','update') +from (select oid from pg_roles where rolname = current_user) as t2; +select has_table_privilege(t2.oid,'pg_authid','delete') +from (select oid from pg_roles where rolname = current_user) as t2; + +-- 'rule' privilege no longer exists, but for backwards compatibility +-- has_table_privilege still recognizes the keyword and says FALSE +select has_table_privilege(current_user,t1.oid,'rule') +from (select oid from pg_class where relname = 'pg_authid') as t1; +select has_table_privilege(current_user,t1.oid,'references') +from (select oid from pg_class where relname = 'pg_authid') as t1; + +select has_table_privilege(t2.oid,t1.oid,'select') +from (select oid from pg_class where relname = 'pg_authid') as t1, + (select oid from pg_roles where rolname = current_user) as t2; +select has_table_privilege(t2.oid,t1.oid,'insert') +from (select oid from pg_class where relname = 'pg_authid') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + +select has_table_privilege('pg_authid','update'); +select has_table_privilege('pg_authid','delete'); +select has_table_privilege('pg_authid','truncate'); + +select has_table_privilege(t1.oid,'select') +from (select oid from pg_class where relname = 'pg_authid') as t1; +select has_table_privilege(t1.oid,'trigger') +from (select oid from pg_class where relname = 'pg_authid') as t1; + +-- non-superuser +SET SESSION AUTHORIZATION regress_priv_user3; + +select has_table_privilege(current_user,'pg_class','select'); +select has_table_privilege(current_user,'pg_class','insert'); + +select has_table_privilege(t2.oid,'pg_class','update') +from (select oid from pg_roles where rolname = current_user) as t2; +select has_table_privilege(t2.oid,'pg_class','delete') +from (select oid from pg_roles where rolname = current_user) as t2; + +select has_table_privilege(current_user,t1.oid,'references') +from (select oid from pg_class where relname = 'pg_class') as t1; + +select has_table_privilege(t2.oid,t1.oid,'select') +from (select oid from pg_class where relname = 'pg_class') as t1, + (select oid from pg_roles where rolname = current_user) as t2; +select has_table_privilege(t2.oid,t1.oid,'insert') +from (select oid from pg_class where relname = 'pg_class') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + +select has_table_privilege('pg_class','update'); +select has_table_privilege('pg_class','delete'); +select has_table_privilege('pg_class','truncate'); + +select has_table_privilege(t1.oid,'select') +from (select oid from pg_class where relname = 'pg_class') as t1; +select has_table_privilege(t1.oid,'trigger') +from (select oid from pg_class where relname = 'pg_class') as t1; + +select has_table_privilege(current_user,'atest1','select'); +select has_table_privilege(current_user,'atest1','insert'); + +select has_table_privilege(t2.oid,'atest1','update') +from (select oid from pg_roles where rolname = current_user) as t2; +select has_table_privilege(t2.oid,'atest1','delete') +from (select oid from pg_roles where rolname = current_user) as t2; + +select has_table_privilege(current_user,t1.oid,'references') +from (select oid from pg_class where relname = 'atest1') as t1; + +select has_table_privilege(t2.oid,t1.oid,'select') +from (select oid from pg_class where relname = 'atest1') as t1, + (select oid from pg_roles where rolname = current_user) as t2; +select has_table_privilege(t2.oid,t1.oid,'insert') +from (select oid from pg_class where relname = 'atest1') as t1, + (select oid from pg_roles where rolname = current_user) as t2; + +select has_table_privilege('atest1','update'); +select has_table_privilege('atest1','delete'); +select has_table_privilege('atest1','truncate'); + +select has_table_privilege(t1.oid,'select') +from (select oid from pg_class where relname = 'atest1') as t1; +select has_table_privilege(t1.oid,'trigger') +from (select oid from pg_class where relname = 'atest1') as t1; + +-- has_column_privilege function + +-- bad-input checks (as non-super-user) +select has_column_privilege('pg_authid',NULL,'select'); +select has_column_privilege('pg_authid','nosuchcol','select'); +select has_column_privilege(9999,'nosuchcol','select'); +select has_column_privilege(9999,99::int2,'select'); +select has_column_privilege('pg_authid',99::int2,'select'); +select has_column_privilege(9999,99::int2,'select'); + +create temp table mytable(f1 int, f2 int, f3 int); +alter table mytable drop column f2; +select has_column_privilege('mytable','f2','select'); +select has_column_privilege('mytable','........pg.dropped.2........','select'); +select has_column_privilege('mytable',2::int2,'select'); +revoke select on table mytable from regress_priv_user3; +select has_column_privilege('mytable',2::int2,'select'); +drop table mytable; + +-- Grant options + +SET SESSION AUTHORIZATION regress_priv_user1; + +CREATE TABLE atest4 (a int); + +GRANT SELECT ON atest4 TO regress_priv_user2 WITH GRANT OPTION; +GRANT UPDATE ON atest4 TO regress_priv_user2; +GRANT SELECT ON atest4 TO GROUP regress_priv_group1 WITH GRANT OPTION; + +SET SESSION AUTHORIZATION regress_priv_user2; + +GRANT SELECT ON atest4 TO regress_priv_user3; +GRANT UPDATE ON atest4 TO regress_priv_user3; -- fail + +SET SESSION AUTHORIZATION regress_priv_user1; + +REVOKE SELECT ON atest4 FROM regress_priv_user3; -- does nothing +SELECT has_table_privilege('regress_priv_user3', 'atest4', 'SELECT'); -- true +REVOKE SELECT ON atest4 FROM regress_priv_user2; -- fail +REVOKE GRANT OPTION FOR SELECT ON atest4 FROM regress_priv_user2 CASCADE; -- ok +SELECT has_table_privilege('regress_priv_user2', 'atest4', 'SELECT'); -- true +SELECT has_table_privilege('regress_priv_user3', 'atest4', 'SELECT'); -- false + +SELECT has_table_privilege('regress_priv_user1', 'atest4', 'SELECT WITH GRANT OPTION'); -- true + +-- Admin options + +SET SESSION AUTHORIZATION regress_priv_user4; +CREATE FUNCTION dogrant_ok() RETURNS void LANGUAGE sql SECURITY DEFINER AS + 'GRANT regress_priv_group2 TO regress_priv_user5'; +GRANT regress_priv_group2 TO regress_priv_user5; -- ok: had ADMIN OPTION +SET ROLE regress_priv_group2; +GRANT regress_priv_group2 TO regress_priv_user5; -- fails: SET ROLE suspended privilege + +SET SESSION AUTHORIZATION regress_priv_user1; +GRANT regress_priv_group2 TO regress_priv_user5; -- fails: no ADMIN OPTION +SELECT dogrant_ok(); -- ok: SECURITY DEFINER conveys ADMIN +SET ROLE regress_priv_group2; +GRANT regress_priv_group2 TO regress_priv_user5; -- fails: SET ROLE did not help + +SET SESSION AUTHORIZATION regress_priv_group2; +GRANT regress_priv_group2 TO regress_priv_user5; -- ok: a role can self-admin +CREATE FUNCTION dogrant_fails() RETURNS void LANGUAGE sql SECURITY DEFINER AS + 'GRANT regress_priv_group2 TO regress_priv_user5'; +SELECT dogrant_fails(); -- fails: no self-admin in SECURITY DEFINER +DROP FUNCTION dogrant_fails(); + +SET SESSION AUTHORIZATION regress_priv_user4; +DROP FUNCTION dogrant_ok(); +REVOKE regress_priv_group2 FROM regress_priv_user5; + +-- has_sequence_privilege tests +\c - + +CREATE SEQUENCE x_seq; + +GRANT USAGE on x_seq to regress_priv_user2; + +SELECT has_sequence_privilege('regress_priv_user1', 'atest1', 'SELECT'); +SELECT has_sequence_privilege('regress_priv_user1', 'x_seq', 'INSERT'); +SELECT has_sequence_privilege('regress_priv_user1', 'x_seq', 'SELECT'); + +SET SESSION AUTHORIZATION regress_priv_user2; + +SELECT has_sequence_privilege('x_seq', 'USAGE'); + +-- largeobject privilege tests +\c - +SET SESSION AUTHORIZATION regress_priv_user1; + +SELECT lo_create(1001); +SELECT lo_create(1002); +SELECT lo_create(1003); +SELECT lo_create(1004); +SELECT lo_create(1005); + +GRANT ALL ON LARGE OBJECT 1001 TO PUBLIC; +GRANT SELECT ON LARGE OBJECT 1003 TO regress_priv_user2; +GRANT SELECT,UPDATE ON LARGE OBJECT 1004 TO regress_priv_user2; +GRANT ALL ON LARGE OBJECT 1005 TO regress_priv_user2; +GRANT SELECT ON LARGE OBJECT 1005 TO regress_priv_user2 WITH GRANT OPTION; + +GRANT SELECT, INSERT ON LARGE OBJECT 1001 TO PUBLIC; -- to be failed +GRANT SELECT, UPDATE ON LARGE OBJECT 1001 TO nosuchuser; -- to be failed +GRANT SELECT, UPDATE ON LARGE OBJECT 999 TO PUBLIC; -- to be failed + +\c - +SET SESSION AUTHORIZATION regress_priv_user2; + +SELECT lo_create(2001); +SELECT lo_create(2002); + +SELECT loread(lo_open(1001, x'20000'::int), 32); -- allowed, for now +SELECT lowrite(lo_open(1001, x'40000'::int), 'abcd'); -- fail, wrong mode + +SELECT loread(lo_open(1001, x'40000'::int), 32); +SELECT loread(lo_open(1002, x'40000'::int), 32); -- to be denied +SELECT loread(lo_open(1003, x'40000'::int), 32); +SELECT loread(lo_open(1004, x'40000'::int), 32); + +SELECT lowrite(lo_open(1001, x'20000'::int), 'abcd'); +SELECT lowrite(lo_open(1002, x'20000'::int), 'abcd'); -- to be denied +SELECT lowrite(lo_open(1003, x'20000'::int), 'abcd'); -- to be denied +SELECT lowrite(lo_open(1004, x'20000'::int), 'abcd'); + +GRANT SELECT ON LARGE OBJECT 1005 TO regress_priv_user3; +GRANT UPDATE ON LARGE OBJECT 1006 TO regress_priv_user3; -- to be denied +REVOKE ALL ON LARGE OBJECT 2001, 2002 FROM PUBLIC; +GRANT ALL ON LARGE OBJECT 2001 TO regress_priv_user3; + +SELECT lo_unlink(1001); -- to be denied +SELECT lo_unlink(2002); + +\c - +-- confirm ACL setting +SELECT oid, pg_get_userbyid(lomowner) ownername, lomacl FROM pg_largeobject_metadata WHERE oid >= 1000 AND oid < 3000 ORDER BY oid; + +SET SESSION AUTHORIZATION regress_priv_user3; + +SELECT loread(lo_open(1001, x'40000'::int), 32); +SELECT loread(lo_open(1003, x'40000'::int), 32); -- to be denied +SELECT loread(lo_open(1005, x'40000'::int), 32); + +SELECT lo_truncate(lo_open(1005, x'20000'::int), 10); -- to be denied +SELECT lo_truncate(lo_open(2001, x'20000'::int), 10); + +-- compatibility mode in largeobject permission +\c - +SET lo_compat_privileges = false; -- default setting +SET SESSION AUTHORIZATION regress_priv_user4; + +SELECT loread(lo_open(1002, x'40000'::int), 32); -- to be denied +SELECT lowrite(lo_open(1002, x'20000'::int), 'abcd'); -- to be denied +SELECT lo_truncate(lo_open(1002, x'20000'::int), 10); -- to be denied +SELECT lo_put(1002, 1, 'abcd'); -- to be denied +SELECT lo_unlink(1002); -- to be denied +SELECT lo_export(1001, '/dev/null'); -- to be denied +SELECT lo_import('/dev/null'); -- to be denied +SELECT lo_import('/dev/null', 2003); -- to be denied + +\c - +SET lo_compat_privileges = true; -- compatibility mode +SET SESSION AUTHORIZATION regress_priv_user4; + +SELECT loread(lo_open(1002, x'40000'::int), 32); +SELECT lowrite(lo_open(1002, x'20000'::int), 'abcd'); +SELECT lo_truncate(lo_open(1002, x'20000'::int), 10); +SELECT lo_unlink(1002); +SELECT lo_export(1001, '/dev/null'); -- to be denied + +-- don't allow unpriv users to access pg_largeobject contents +\c - +SELECT * FROM pg_largeobject LIMIT 0; + +SET SESSION AUTHORIZATION regress_priv_user1; +SELECT * FROM pg_largeobject LIMIT 0; -- to be denied + +-- test default ACLs +\c - + +CREATE SCHEMA testns; +GRANT ALL ON SCHEMA testns TO regress_priv_user1; + +CREATE TABLE testns.acltest1 (x int); +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- no +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT SELECT ON TABLES TO public; + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- no +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + +DROP TABLE testns.acltest1; +CREATE TABLE testns.acltest1 (x int); + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- yes +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT INSERT ON TABLES TO regress_priv_user1; + +DROP TABLE testns.acltest1; +CREATE TABLE testns.acltest1 (x int); + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- yes +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- yes + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns REVOKE INSERT ON TABLES FROM regress_priv_user1; + +DROP TABLE testns.acltest1; +CREATE TABLE testns.acltest1 (x int); + +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'SELECT'); -- yes +SELECT has_table_privilege('regress_priv_user1', 'testns.acltest1', 'INSERT'); -- no + +ALTER DEFAULT PRIVILEGES FOR ROLE regress_priv_user1 REVOKE EXECUTE ON FUNCTIONS FROM public; + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT USAGE ON SCHEMAS TO regress_priv_user2; -- error + +-- +-- Testing blanket default grants is very hazardous since it might change +-- the privileges attached to objects created by concurrent regression tests. +-- To avoid that, be sure to revoke the privileges again before committing. +-- +BEGIN; + +ALTER DEFAULT PRIVILEGES GRANT USAGE ON SCHEMAS TO regress_priv_user2; + +CREATE SCHEMA testns2; + +SELECT has_schema_privilege('regress_priv_user2', 'testns2', 'USAGE'); -- yes +SELECT has_schema_privilege('regress_priv_user2', 'testns2', 'CREATE'); -- no + +ALTER DEFAULT PRIVILEGES REVOKE USAGE ON SCHEMAS FROM regress_priv_user2; + +CREATE SCHEMA testns3; + +SELECT has_schema_privilege('regress_priv_user2', 'testns3', 'USAGE'); -- no +SELECT has_schema_privilege('regress_priv_user2', 'testns3', 'CREATE'); -- no + +ALTER DEFAULT PRIVILEGES GRANT ALL ON SCHEMAS TO regress_priv_user2; + +CREATE SCHEMA testns4; + +SELECT has_schema_privilege('regress_priv_user2', 'testns4', 'USAGE'); -- yes +SELECT has_schema_privilege('regress_priv_user2', 'testns4', 'CREATE'); -- yes + +ALTER DEFAULT PRIVILEGES REVOKE ALL ON SCHEMAS FROM regress_priv_user2; + +COMMIT; + +CREATE SCHEMA testns5; + +SELECT has_schema_privilege('regress_priv_user2', 'testns5', 'USAGE'); -- no +SELECT has_schema_privilege('regress_priv_user2', 'testns5', 'CREATE'); -- no + +SET ROLE regress_priv_user1; + +CREATE FUNCTION testns.foo() RETURNS int AS 'select 1' LANGUAGE sql; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE AGGREGATE testns.agg1(int) (sfunc = int4pl, stype = int4); +-- +CREATE PROCEDURE testns.bar() AS 'select 1' LANGUAGE sql; + +SELECT has_function_privilege('regress_priv_user2', 'testns.foo()', 'EXECUTE'); -- no +-- SELECT has_function_privilege('regress_priv_user2', 'testns.agg1(int)', 'EXECUTE'); -- no +SELECT has_function_privilege('regress_priv_user2', 'testns.bar()', 'EXECUTE'); -- no + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT EXECUTE ON ROUTINES to public; + +DROP FUNCTION testns.foo(); +CREATE FUNCTION testns.foo() RETURNS int AS 'select 1' LANGUAGE sql; +-- DROP AGGREGATE testns.agg1(int); +-- CREATE AGGREGATE testns.agg1(int) (sfunc = int4pl, stype = int4); +DROP PROCEDURE testns.bar(); +CREATE PROCEDURE testns.bar() AS 'select 1' LANGUAGE sql; + +SELECT has_function_privilege('regress_priv_user2', 'testns.foo()', 'EXECUTE'); -- yes +-- SELECT has_function_privilege('regress_priv_user2', 'testns.agg1(int)', 'EXECUTE'); -- yes +SELECT has_function_privilege('regress_priv_user2', 'testns.bar()', 'EXECUTE'); -- yes (counts as function here) + +DROP FUNCTION testns.foo(); +-- DROP AGGREGATE testns.agg1(int); +DROP PROCEDURE testns.bar(); + +ALTER DEFAULT PRIVILEGES FOR ROLE regress_priv_user1 REVOKE USAGE ON TYPES FROM public; + +CREATE DOMAIN testns.priv_testdomain1 AS int; + +SELECT has_type_privilege('regress_priv_user2', 'testns.priv_testdomain1', 'USAGE'); -- no + +ALTER DEFAULT PRIVILEGES IN SCHEMA testns GRANT USAGE ON TYPES to public; + +DROP DOMAIN testns.priv_testdomain1; +CREATE DOMAIN testns.priv_testdomain1 AS int; + +SELECT has_type_privilege('regress_priv_user2', 'testns.priv_testdomain1', 'USAGE'); -- yes + +DROP DOMAIN testns.priv_testdomain1; + +RESET ROLE; + +SELECT count(*) + FROM pg_default_acl d LEFT JOIN pg_namespace n ON defaclnamespace = n.oid + WHERE nspname = 'testns'; + +DROP SCHEMA testns CASCADE; +DROP SCHEMA testns2 CASCADE; +DROP SCHEMA testns4 CASCADE; +DROP SCHEMA testns5 CASCADE; + +SELECT d.* -- check that entries went away + FROM pg_default_acl d LEFT JOIN pg_namespace n ON defaclnamespace = n.oid + WHERE nspname IS NULL AND defaclnamespace != 0; + + +-- Grant on all objects of given type in a schema +\c - + +CREATE SCHEMA testns; +CREATE TABLE testns.t1 (f1 int); +CREATE TABLE testns.t2 (f1 int); + +SELECT has_table_privilege('regress_priv_user1', 'testns.t1', 'SELECT'); -- false + +GRANT ALL ON ALL TABLES IN SCHEMA testns TO regress_priv_user1; + +SELECT has_table_privilege('regress_priv_user1', 'testns.t1', 'SELECT'); -- true +SELECT has_table_privilege('regress_priv_user1', 'testns.t2', 'SELECT'); -- true + +REVOKE ALL ON ALL TABLES IN SCHEMA testns FROM regress_priv_user1; + +SELECT has_table_privilege('regress_priv_user1', 'testns.t1', 'SELECT'); -- false +SELECT has_table_privilege('regress_priv_user1', 'testns.t2', 'SELECT'); -- false + +CREATE FUNCTION testns.priv_testfunc(int) RETURNS int AS 'select 3 * $1;' LANGUAGE sql; +-- NOT SUPPORTED +-- +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE AGGREGATE testns.priv_testagg(int) (sfunc = int4pl, stype = int4); +-- +CREATE PROCEDURE testns.priv_testproc(int) AS 'select 3' LANGUAGE sql; + +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testfunc(int)', 'EXECUTE'); -- true by default +-- SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testagg(int)', 'EXECUTE'); -- true by default +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- true by default + +REVOKE ALL ON ALL FUNCTIONS IN SCHEMA testns FROM PUBLIC; + +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testfunc(int)', 'EXECUTE'); -- false +-- SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testagg(int)', 'EXECUTE'); -- false +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- still true, not a function + +REVOKE ALL ON ALL PROCEDURES IN SCHEMA testns FROM PUBLIC; + +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- now false + +GRANT ALL ON ALL ROUTINES IN SCHEMA testns TO PUBLIC; + +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testfunc(int)', 'EXECUTE'); -- true +-- SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testagg(int)', 'EXECUTE'); -- true +SELECT has_function_privilege('regress_priv_user1', 'testns.priv_testproc(int)', 'EXECUTE'); -- true + + +\set VERBOSITY terse \\ -- suppress cascade details +DROP SCHEMA testns CASCADE; +\set VERBOSITY default + +-- Change owner of the schema & and rename of new schema owner +\c - + +CREATE ROLE regress_schemauser1 superuser login; +CREATE ROLE regress_schemauser2 superuser login; + +SET SESSION ROLE regress_schemauser1; +CREATE SCHEMA testns; + +SELECT nspname, rolname FROM pg_namespace, pg_roles WHERE pg_namespace.nspname = 'testns' AND pg_namespace.nspowner = pg_roles.oid; + +ALTER SCHEMA testns OWNER TO regress_schemauser2; +ALTER ROLE regress_schemauser2 RENAME TO regress_schemauser_renamed; +SELECT nspname, rolname FROM pg_namespace, pg_roles WHERE pg_namespace.nspname = 'testns' AND pg_namespace.nspowner = pg_roles.oid; + +set session role regress_schemauser_renamed; +\set VERBOSITY terse \\ -- suppress cascade details +DROP SCHEMA testns CASCADE; +\set VERBOSITY default + +-- clean up +\c - + +DROP ROLE regress_schemauser1; +DROP ROLE regress_schemauser_renamed; + +-- test that dependent privileges are revoked (or not) properly +\c - + +set session role regress_priv_user1; +create table dep_priv_test (a int); +grant select on dep_priv_test to regress_priv_user2 with grant option; +grant select on dep_priv_test to regress_priv_user3 with grant option; +set session role regress_priv_user2; +grant select on dep_priv_test to regress_priv_user4 with grant option; +set session role regress_priv_user3; +grant select on dep_priv_test to regress_priv_user4 with grant option; +set session role regress_priv_user4; +grant select on dep_priv_test to regress_priv_user5; +\dp dep_priv_test +set session role regress_priv_user2; +revoke select on dep_priv_test from regress_priv_user4 cascade; +\dp dep_priv_test +set session role regress_priv_user3; +revoke select on dep_priv_test from regress_priv_user4 cascade; +\dp dep_priv_test +set session role regress_priv_user1; +drop table dep_priv_test; + + +-- clean up + +\c + +drop sequence x_seq; + +-- DROP AGGREGATE priv_testagg1(int); +DROP FUNCTION priv_testfunc2(int); +DROP FUNCTION priv_testfunc4(boolean); +DROP PROCEDURE priv_testproc1(int); + +DROP VIEW atestv0; +DROP VIEW atestv1; +DROP VIEW atestv2; +-- this should cascade to drop atestv4 +DROP VIEW atestv3 CASCADE; +-- this should complain "does not exist" +DROP VIEW atestv4; + +DROP TABLE atest1; +DROP TABLE atest2; +DROP TABLE atest3; +DROP TABLE atest4; +DROP TABLE atest5; +DROP TABLE atest6; + +DROP TABLE priv_int8_tbl; + +DROP SEQUENCE twoseq; +DROP SEQUENCE fourseq; + +SELECT lo_unlink(oid) FROM pg_largeobject_metadata WHERE oid >= 1000 AND oid < 3000 ORDER BY oid; + +DROP GROUP regress_priv_group1; +DROP GROUP regress_priv_group2; + +-- these are needed to clean up permissions +REVOKE USAGE ON LANGUAGE sql FROM regress_priv_user1; +DROP OWNED BY regress_priv_user1; + +DROP USER regress_priv_user1; +DROP USER regress_priv_user2; +DROP USER regress_priv_user3; +DROP USER regress_priv_user4; +DROP USER regress_priv_user5; +DROP USER regress_priv_user6; + +-- NOT SUPPORTED +-- +-- -- permissions with LOCK TABLE +-- CREATE USER regress_locktable_user; +-- CREATE TABLE lock_table (a int); +-- +-- -- LOCK TABLE and SELECT permission +-- GRANT SELECT ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should fail +-- ROLLBACK; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS SHARE MODE; -- should pass +-- COMMIT; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should fail +-- ROLLBACK; +-- \c +-- REVOKE SELECT ON lock_table FROM regress_locktable_user; +-- +-- -- LOCK TABLE and INSERT permission +-- GRANT INSERT ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- BEGIN;BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- \c +-- REVOKE UPDATE ON lock_table FROM regress_locktable_user; +-- +-- -- LOCK TABLE and DELETE permission +-- GRANT DELETE ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS SHARE MODE; -- should fail +-- ROLLBACK; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- \c +-- REVOKE DELETE ON lock_table FROM regress_locktable_user; +-- +-- -- LOCK TABLE and TRUNCATE permission +-- GRANT TRUNCATE ON lock_table TO regress_locktable_user; +-- SET SESSION AUTHORIZATION regress_locktable_user; +-- BEGIN; +-- LOCK TABLE lock_table IN ROW EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS SHARE MODE; -- should fail +-- ROLLBACK; +-- BEGIN; +-- LOCK TABLE lock_table IN ACCESS EXCLUSIVE MODE; -- should pass +-- COMMIT; +-- \c +-- REVOKE TRUNCATE ON lock_table FROM regress_locktable_user; +-- +-- -- clean up +-- DROP TABLE lock_table; +-- DROP USER regress_locktable_user; +-- diff --git a/src/postgres/src/test/regress/sql/yb_roleattributes.sql b/src/postgres/src/test/regress/sql/yb_roleattributes.sql new file mode 100644 index 000000000000..1b034d752fb8 --- /dev/null +++ b/src/postgres/src/test/regress/sql/yb_roleattributes.sql @@ -0,0 +1,97 @@ +-- default for superuser is false +CREATE ROLE regress_test_def_superuser; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_superuser'; +CREATE ROLE regress_test_superuser WITH SUPERUSER; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_superuser'; +ALTER ROLE regress_test_superuser WITH NOSUPERUSER; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_superuser'; +ALTER ROLE regress_test_superuser WITH SUPERUSER; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_superuser'; + +-- default for inherit is true +CREATE ROLE regress_test_def_inherit; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_inherit'; +CREATE ROLE regress_test_inherit WITH NOINHERIT; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_inherit'; +ALTER ROLE regress_test_inherit WITH INHERIT; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_inherit'; +ALTER ROLE regress_test_inherit WITH NOINHERIT; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_inherit'; + +-- default for create role is false +CREATE ROLE regress_test_def_createrole; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_createrole'; +CREATE ROLE regress_test_createrole WITH CREATEROLE; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createrole'; +ALTER ROLE regress_test_createrole WITH NOCREATEROLE; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createrole'; +ALTER ROLE regress_test_createrole WITH CREATEROLE; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createrole'; + +-- default for create database is false +CREATE ROLE regress_test_def_createdb; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_createdb'; +CREATE ROLE regress_test_createdb WITH CREATEDB; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createdb'; +ALTER ROLE regress_test_createdb WITH NOCREATEDB; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createdb'; +ALTER ROLE regress_test_createdb WITH CREATEDB; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_createdb'; + +-- default for can login is false for role +CREATE ROLE regress_test_def_role_canlogin; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_role_canlogin'; +CREATE ROLE regress_test_role_canlogin WITH LOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_role_canlogin'; +ALTER ROLE regress_test_role_canlogin WITH NOLOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_role_canlogin'; +ALTER ROLE regress_test_role_canlogin WITH LOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_role_canlogin'; + +-- default for can login is true for user +CREATE USER regress_test_def_user_canlogin; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_user_canlogin'; +CREATE USER regress_test_user_canlogin WITH NOLOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_user_canlogin'; +ALTER USER regress_test_user_canlogin WITH LOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_user_canlogin'; +ALTER USER regress_test_user_canlogin WITH NOLOGIN; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_user_canlogin'; + +-- default for replication is false +CREATE ROLE regress_test_def_replication; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_replication'; +CREATE ROLE regress_test_replication WITH REPLICATION; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_replication'; +ALTER ROLE regress_test_replication WITH NOREPLICATION; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_replication'; +ALTER ROLE regress_test_replication WITH REPLICATION; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_replication'; + +-- default for bypassrls is false +CREATE ROLE regress_test_def_bypassrls; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_def_bypassrls'; +CREATE ROLE regress_test_bypassrls WITH BYPASSRLS; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_bypassrls'; +ALTER ROLE regress_test_bypassrls WITH NOBYPASSRLS; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_bypassrls'; +ALTER ROLE regress_test_bypassrls WITH BYPASSRLS; +SELECT * FROM pg_authid WHERE rolname = 'regress_test_bypassrls'; + +-- clean up roles +DROP ROLE regress_test_def_superuser; +DROP ROLE regress_test_superuser; +DROP ROLE regress_test_def_inherit; +DROP ROLE regress_test_inherit; +DROP ROLE regress_test_def_createrole; +DROP ROLE regress_test_createrole; +DROP ROLE regress_test_def_createdb; +DROP ROLE regress_test_createdb; +DROP ROLE regress_test_def_role_canlogin; +DROP ROLE regress_test_role_canlogin; +DROP USER regress_test_def_user_canlogin; +DROP USER regress_test_user_canlogin; +DROP ROLE regress_test_def_replication; +DROP ROLE regress_test_replication; +DROP ROLE regress_test_def_bypassrls; +DROP ROLE regress_test_bypassrls; diff --git a/src/postgres/src/test/regress/sql/yb_rolenames.sql b/src/postgres/src/test/regress/sql/yb_rolenames.sql new file mode 100644 index 000000000000..54cf6c08b21c --- /dev/null +++ b/src/postgres/src/test/regress/sql/yb_rolenames.sql @@ -0,0 +1,457 @@ +CREATE OR REPLACE FUNCTION chkrolattr() + RETURNS TABLE ("role" name, rolekeyword text, canlogin bool, replication bool) + AS $$ +SELECT r.rolname, v.keyword, r.rolcanlogin, r.rolreplication + FROM pg_roles r + JOIN (VALUES(CURRENT_USER, 'current_user'), + (SESSION_USER, 'session_user'), + ('current_user', '-'), + ('session_user', '-'), + ('Public', '-'), + ('None', '-')) + AS v(uname, keyword) + ON (r.rolname = v.uname) + ORDER BY 1; +$$ LANGUAGE SQL; + +CREATE OR REPLACE FUNCTION chksetconfig() + RETURNS TABLE (db name, "role" name, rolkeyword text, setconfig text[]) + AS $$ +SELECT COALESCE(d.datname, 'ALL'), COALESCE(r.rolname, 'ALL'), + COALESCE(v.keyword, '-'), s.setconfig + FROM pg_db_role_setting s + LEFT JOIN pg_roles r ON (r.oid = s.setrole) + LEFT JOIN pg_database d ON (d.oid = s.setdatabase) + LEFT JOIN (VALUES(CURRENT_USER, 'current_user'), + (SESSION_USER, 'session_user')) + AS v(uname, keyword) + ON (r.rolname = v.uname) + WHERE (r.rolname) IN ('Public', 'current_user', 'regress_testrol1', 'regress_testrol2') +ORDER BY 1, 2; +$$ LANGUAGE SQL; + +CREATE OR REPLACE FUNCTION chkumapping() + RETURNS TABLE (umname name, umserver name, umoptions text[]) + AS $$ +SELECT r.rolname, s.srvname, m.umoptions + FROM pg_user_mapping m + LEFT JOIN pg_roles r ON (r.oid = m.umuser) + JOIN pg_foreign_server s ON (s.oid = m.umserver) + ORDER BY 2; +$$ LANGUAGE SQL; + +CREATE ROLE "Public"; +CREATE ROLE "None"; +CREATE ROLE "current_user"; +CREATE ROLE "session_user"; +CREATE ROLE "user"; + +CREATE ROLE current_user; -- error +CREATE ROLE current_role; -- error +CREATE ROLE session_user; -- error +CREATE ROLE user; -- error +CREATE ROLE all; -- error + +CREATE ROLE public; -- error +CREATE ROLE "public"; -- error +CREATE ROLE none; -- error +CREATE ROLE "none"; -- error + +CREATE ROLE pg_abc; -- error +CREATE ROLE "pg_abc"; -- error +CREATE ROLE pg_abcdef; -- error +CREATE ROLE "pg_abcdef"; -- error + +CREATE ROLE regress_testrol0 SUPERUSER LOGIN; +CREATE ROLE regress_testrolx SUPERUSER LOGIN; +CREATE ROLE regress_testrol2 SUPERUSER; +CREATE ROLE regress_testrol1 SUPERUSER LOGIN IN ROLE regress_testrol2; + +\c - +SET SESSION AUTHORIZATION regress_testrol1; +SET ROLE regress_testrol2; + +-- ALTER ROLE +SELECT * FROM chkrolattr(); +ALTER ROLE CURRENT_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER ROLE "current_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER ROLE SESSION_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER ROLE "session_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER "Public" WITH REPLICATION; +ALTER USER "None" WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER regress_testrol1 WITH NOREPLICATION; +ALTER USER regress_testrol2 WITH NOREPLICATION; +SELECT * FROM chkrolattr(); + +-- Manually rollback the above changes +-- TODO put this in a transaction after #1383 + +ALTER ROLE "None" NOREPLICATION; +ALTER ROLE "Public" NOREPLICATION; +ALTER ROLE "current_user" NOREPLICATION; +ALTER ROLE "session_user" NOREPLICATION; + +ALTER ROLE USER WITH LOGIN; -- error +ALTER ROLE CURRENT_ROLE WITH LOGIN; --error +ALTER ROLE ALL WITH REPLICATION; -- error +ALTER ROLE SESSION_ROLE WITH NOREPLICATION; -- error +ALTER ROLE PUBLIC WITH NOREPLICATION; -- error +ALTER ROLE "public" WITH NOREPLICATION; -- error +ALTER ROLE NONE WITH NOREPLICATION; -- error +ALTER ROLE "none" WITH NOREPLICATION; -- error +ALTER ROLE nonexistent WITH NOREPLICATION; -- error + +-- ALTER USER +BEGIN; +SELECT * FROM chkrolattr(); +ALTER USER CURRENT_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER "current_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER SESSION_USER WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER "session_user" WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER "Public" WITH REPLICATION; +ALTER USER "None" WITH REPLICATION; +SELECT * FROM chkrolattr(); +ALTER USER regress_testrol1 WITH NOREPLICATION; +ALTER USER regress_testrol2 WITH NOREPLICATION; +SELECT * FROM chkrolattr(); +ROLLBACK; + +ALTER USER USER WITH LOGIN; -- error +ALTER USER CURRENT_ROLE WITH LOGIN; -- error +ALTER USER ALL WITH REPLICATION; -- error +ALTER USER SESSION_ROLE WITH NOREPLICATION; -- error +ALTER USER PUBLIC WITH NOREPLICATION; -- error +ALTER USER "public" WITH NOREPLICATION; -- error +ALTER USER NONE WITH NOREPLICATION; -- error +ALTER USER "none" WITH NOREPLICATION; -- error +ALTER USER nonexistent WITH NOREPLICATION; -- error + +-- ALTER ROLE SET/RESET +SELECT * FROM chksetconfig(); +ALTER ROLE CURRENT_USER SET application_name to 'FOO'; +ALTER ROLE SESSION_USER SET application_name to 'BAR'; +ALTER ROLE "current_user" SET application_name to 'FOOFOO'; +ALTER ROLE "Public" SET application_name to 'BARBAR'; +ALTER ROLE ALL SET application_name to 'SLAP'; +SELECT * FROM chksetconfig(); +ALTER ROLE regress_testrol1 SET application_name to 'SLAM'; +SELECT * FROM chksetconfig(); +ALTER ROLE CURRENT_USER RESET application_name; +ALTER ROLE SESSION_USER RESET application_name; +ALTER ROLE "current_user" RESET application_name; +ALTER ROLE "Public" RESET application_name; +ALTER ROLE ALL RESET application_name; +SELECT * FROM chksetconfig(); + + +ALTER ROLE CURRENT_ROLE SET application_name to 'BAZ'; -- error +ALTER ROLE USER SET application_name to 'BOOM'; -- error +ALTER ROLE PUBLIC SET application_name to 'BOMB'; -- error +ALTER ROLE nonexistent SET application_name to 'BOMB'; -- error + +-- ALTER USER SET/RESET +SELECT * FROM chksetconfig(); +ALTER USER CURRENT_USER SET application_name to 'FOO'; +ALTER USER SESSION_USER SET application_name to 'BAR'; +ALTER USER "current_user" SET application_name to 'FOOFOO'; +ALTER USER "Public" SET application_name to 'BARBAR'; +ALTER USER ALL SET application_name to 'SLAP'; +SELECT * FROM chksetconfig(); +ALTER USER regress_testrol1 SET application_name to 'SLAM'; +SELECT * FROM chksetconfig(); +ALTER USER CURRENT_USER RESET application_name; +ALTER USER SESSION_USER RESET application_name; +ALTER USER "current_user" RESET application_name; +ALTER USER "Public" RESET application_name; +ALTER USER ALL RESET application_name; +SELECT * FROM chksetconfig(); + + +ALTER USER CURRENT_USER SET application_name to 'BAZ'; -- error +ALTER USER USER SET application_name to 'BOOM'; -- error +ALTER USER PUBLIC SET application_name to 'BOMB'; -- error +ALTER USER NONE SET application_name to 'BOMB'; -- error +ALTER USER nonexistent SET application_name to 'BOMB'; -- error + +-- CREATE SCHEMA +set client_min_messages to error; +CREATE SCHEMA newschema1 AUTHORIZATION CURRENT_USER; +CREATE SCHEMA newschema2 AUTHORIZATION "current_user"; +CREATE SCHEMA newschema3 AUTHORIZATION SESSION_USER; +CREATE SCHEMA newschema4 AUTHORIZATION regress_testrolx; +CREATE SCHEMA newschema5 AUTHORIZATION "Public"; + +CREATE SCHEMA newschema6 AUTHORIZATION USER; -- error +CREATE SCHEMA newschema6 AUTHORIZATION CURRENT_ROLE; -- error +CREATE SCHEMA newschema6 AUTHORIZATION PUBLIC; -- error +CREATE SCHEMA newschema6 AUTHORIZATION "public"; -- error +CREATE SCHEMA newschema6 AUTHORIZATION NONE; -- error +CREATE SCHEMA newschema6 AUTHORIZATION nonexistent; -- error + +SELECT n.nspname, r.rolname FROM pg_namespace n + JOIN pg_roles r ON (r.oid = n.nspowner) + WHERE n.nspname LIKE 'newschema_' ORDER BY 1; + +CREATE SCHEMA IF NOT EXISTS newschema1 AUTHORIZATION CURRENT_USER; +CREATE SCHEMA IF NOT EXISTS newschema2 AUTHORIZATION "current_user"; +CREATE SCHEMA IF NOT EXISTS newschema3 AUTHORIZATION SESSION_USER; +CREATE SCHEMA IF NOT EXISTS newschema4 AUTHORIZATION regress_testrolx; +CREATE SCHEMA IF NOT EXISTS newschema5 AUTHORIZATION "Public"; + +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION USER; -- error +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION CURRENT_ROLE; -- error +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION PUBLIC; -- error +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION "public"; -- error +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION NONE; -- error +CREATE SCHEMA IF NOT EXISTS newschema6 AUTHORIZATION nonexistent; -- error + +SELECT n.nspname, r.rolname FROM pg_namespace n + JOIN pg_roles r ON (r.oid = n.nspowner) + WHERE n.nspname LIKE 'newschema_' ORDER BY 1; + +-- ALTER TABLE OWNER TO +\c - +SET SESSION AUTHORIZATION regress_testrol0; +set client_min_messages to error; +CREATE TABLE testtab1 (a int); +CREATE TABLE testtab2 (a int); +CREATE TABLE testtab3 (a int); +CREATE TABLE testtab4 (a int); +CREATE TABLE testtab5 (a int); +CREATE TABLE testtab6 (a int); + +\c - +SET SESSION AUTHORIZATION regress_testrol1; +SET ROLE regress_testrol2; + +ALTER TABLE testtab1 OWNER TO CURRENT_USER; +ALTER TABLE testtab2 OWNER TO "current_user"; +ALTER TABLE testtab3 OWNER TO SESSION_USER; +ALTER TABLE testtab4 OWNER TO regress_testrolx; +ALTER TABLE testtab5 OWNER TO "Public"; + +ALTER TABLE testtab6 OWNER TO CURRENT_ROLE; -- error +ALTER TABLE testtab6 OWNER TO USER; --error +ALTER TABLE testtab6 OWNER TO PUBLIC; -- error +ALTER TABLE testtab6 OWNER TO "public"; -- error +ALTER TABLE testtab6 OWNER TO nonexistent; -- error + +SELECT c.relname, r.rolname + FROM pg_class c JOIN pg_roles r ON (r.oid = c.relowner) + WHERE relname LIKE 'testtab_' + ORDER BY 1; + +-- ALTER TABLE, VIEW, MATERIALIZED VIEW, FOREIGN TABLE, SEQUENCE are +-- changed their owner in the same way. + +-- ALTER FUNCTION +\c - +SET SESSION AUTHORIZATION regress_testrol0; +CREATE FUNCTION testagg1(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg2(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg3(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg4(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg5(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg5(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg6(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg7(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg8(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; +CREATE FUNCTION testagg9(int2) RETURNS int AS $$ SELECT 1 $$ LANGUAGE SQL; + +\c - +SET SESSION AUTHORIZATION regress_testrol1; +SET ROLE regress_testrol2; + +ALTER FUNCTION testagg1(int2) OWNER TO CURRENT_USER; +ALTER FUNCTION testagg2(int2) OWNER TO "current_user"; +ALTER FUNCTION testagg3(int2) OWNER TO SESSION_USER; +ALTER FUNCTION testagg4(int2) OWNER TO regress_testrolx; +ALTER FUNCTION testagg5(int2) OWNER TO "Public"; + +ALTER FUNCTION testagg5(int2) OWNER TO CURRENT_ROLE; -- error +ALTER FUNCTION testagg5(int2) OWNER TO USER; -- error +ALTER FUNCTION testagg5(int2) OWNER TO PUBLIC; -- error +ALTER FUNCTION testagg5(int2) OWNER TO "public"; -- error +ALTER FUNCTION testagg5(int2) OWNER TO nonexistent; -- error + +SELECT p.proname, r.rolname + FROM pg_proc p JOIN pg_roles r ON (r.oid = p.proowner) + WHERE proname LIKE 'testagg_' + ORDER BY 1; + +-- NOT SUPPORTED +-- +-- -- CREATE USER MAPPING +-- CREATE FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv1 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv2 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv3 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv4 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv5 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv6 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv7 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv8 FOREIGN DATA WRAPPER test_wrapper; +-- CREATE SERVER sv9 FOREIGN DATA WRAPPER test_wrapper; +-- +-- -- IF THIS LINE CAUSES A FAILURE, THIS REGION MAY BE SUPPORTED +CREATE USER MAPPING FOR CURRENT_USER SERVER sv1 OPTIONS (user 'CURRENT_USER'); +-- CREATE USER MAPPING FOR "current_user" SERVER sv2 OPTIONS (user '"current_user"'); +-- CREATE USER MAPPING FOR USER SERVER sv3 OPTIONS (user 'USER'); +-- CREATE USER MAPPING FOR "user" SERVER sv4 OPTIONS (user '"USER"'); +-- CREATE USER MAPPING FOR SESSION_USER SERVER sv5 OPTIONS (user 'SESSION_USER'); +-- CREATE USER MAPPING FOR PUBLIC SERVER sv6 OPTIONS (user 'PUBLIC'); +-- CREATE USER MAPPING FOR "Public" SERVER sv7 OPTIONS (user '"Public"'); +-- CREATE USER MAPPING FOR regress_testrolx SERVER sv8 OPTIONS (user 'regress_testrolx'); +-- +-- CREATE USER MAPPING FOR CURRENT_ROLE SERVER sv9 +-- OPTIONS (user 'CURRENT_ROLE'); -- error +-- CREATE USER MAPPING FOR nonexistent SERVER sv9 +-- OPTIONS (user 'nonexistent'); -- error; +-- +-- SELECT * FROM chkumapping(); +-- +-- -- ALTER USER MAPPING +-- ALTER USER MAPPING FOR CURRENT_USER SERVER sv1 +-- OPTIONS (SET user 'CURRENT_USER_alt'); +-- ALTER USER MAPPING FOR "current_user" SERVER sv2 +-- OPTIONS (SET user '"current_user"_alt'); +-- ALTER USER MAPPING FOR USER SERVER sv3 +-- OPTIONS (SET user 'USER_alt'); +-- ALTER USER MAPPING FOR "user" SERVER sv4 +-- OPTIONS (SET user '"user"_alt'); +-- ALTER USER MAPPING FOR SESSION_USER SERVER sv5 +-- OPTIONS (SET user 'SESSION_USER_alt'); +-- ALTER USER MAPPING FOR PUBLIC SERVER sv6 +-- OPTIONS (SET user 'public_alt'); +-- ALTER USER MAPPING FOR "Public" SERVER sv7 +-- OPTIONS (SET user '"Public"_alt'); +-- ALTER USER MAPPING FOR regress_testrolx SERVER sv8 +-- OPTIONS (SET user 'regress_testrolx_alt'); +-- +-- ALTER USER MAPPING FOR CURRENT_ROLE SERVER sv9 +-- OPTIONS (SET user 'CURRENT_ROLE_alt'); +-- ALTER USER MAPPING FOR nonexistent SERVER sv9 +-- OPTIONS (SET user 'nonexistent_alt'); -- error +-- +-- SELECT * FROM chkumapping(); +-- +-- -- DROP USER MAPPING +-- DROP USER MAPPING FOR CURRENT_USER SERVER sv1; +-- DROP USER MAPPING FOR "current_user" SERVER sv2; +-- DROP USER MAPPING FOR USER SERVER sv3; +-- DROP USER MAPPING FOR "user" SERVER sv4; +-- DROP USER MAPPING FOR SESSION_USER SERVER sv5; +-- DROP USER MAPPING FOR PUBLIC SERVER sv6; +-- DROP USER MAPPING FOR "Public" SERVER sv7; +-- DROP USER MAPPING FOR regress_testrolx SERVER sv8; +-- +-- DROP USER MAPPING FOR CURRENT_ROLE SERVER sv9; -- error +-- DROP USER MAPPING FOR nonexistent SERVER sv; -- error +-- SELECT * FROM chkumapping(); +-- +-- CREATE USER MAPPING FOR CURRENT_USER SERVER sv1 OPTIONS (user 'CURRENT_USER'); +-- CREATE USER MAPPING FOR "current_user" SERVER sv2 OPTIONS (user '"current_user"'); +-- CREATE USER MAPPING FOR USER SERVER sv3 OPTIONS (user 'USER'); +-- CREATE USER MAPPING FOR "user" SERVER sv4 OPTIONS (user '"USER"'); +-- CREATE USER MAPPING FOR SESSION_USER SERVER sv5 OPTIONS (user 'SESSION_USER'); +-- CREATE USER MAPPING FOR PUBLIC SERVER sv6 OPTIONS (user 'PUBLIC'); +-- CREATE USER MAPPING FOR "Public" SERVER sv7 OPTIONS (user '"Public"'); +-- CREATE USER MAPPING FOR regress_testrolx SERVER sv8 OPTIONS (user 'regress_testrolx'); +-- SELECT * FROM chkumapping(); +-- +-- -- DROP USER MAPPING IF EXISTS +-- DROP USER MAPPING IF EXISTS FOR CURRENT_USER SERVER sv1; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR "current_user" SERVER sv2; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR USER SERVER sv3; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR "user" SERVER sv4; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR SESSION_USER SERVER sv5; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR PUBLIC SERVER sv6; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR "Public" SERVER sv7; +-- SELECT * FROM chkumapping(); +-- DROP USER MAPPING IF EXISTS FOR regress_testrolx SERVER sv8; +-- SELECT * FROM chkumapping(); +-- +-- DROP USER MAPPING IF EXISTS FOR CURRENT_ROLE SERVER sv9; --error +-- DROP USER MAPPING IF EXISTS FOR nonexistent SERVER sv9; -- error +-- + +-- GRANT/REVOKE +GRANT regress_testrol0 TO pg_signal_backend; -- success + +SET ROLE pg_signal_backend; --success +RESET ROLE; +CREATE SCHEMA test_roles_schema AUTHORIZATION pg_signal_backend; --success +SET ROLE regress_testrol2; + +UPDATE pg_proc SET proacl = null WHERE proname LIKE 'testagg_'; +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + +REVOKE ALL PRIVILEGES ON FUNCTION testagg1(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg2(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg3(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg4(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg5(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg6(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg7(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg8(int2) FROM PUBLIC; + +GRANT ALL PRIVILEGES ON FUNCTION testagg1(int2) TO PUBLIC; +GRANT ALL PRIVILEGES ON FUNCTION testagg2(int2) TO CURRENT_USER; +GRANT ALL PRIVILEGES ON FUNCTION testagg3(int2) TO "current_user"; +GRANT ALL PRIVILEGES ON FUNCTION testagg4(int2) TO SESSION_USER; +GRANT ALL PRIVILEGES ON FUNCTION testagg5(int2) TO "Public"; +GRANT ALL PRIVILEGES ON FUNCTION testagg6(int2) TO regress_testrolx; +GRANT ALL PRIVILEGES ON FUNCTION testagg7(int2) TO "public"; +GRANT ALL PRIVILEGES ON FUNCTION testagg8(int2) + TO current_user, public, regress_testrolx; + +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO CURRENT_ROLE; --error +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO USER; --error +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO NONE; --error +GRANT ALL PRIVILEGES ON FUNCTION testagg9(int2) TO "none"; --error + +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + +REVOKE ALL PRIVILEGES ON FUNCTION testagg1(int2) FROM PUBLIC; +REVOKE ALL PRIVILEGES ON FUNCTION testagg2(int2) FROM CURRENT_USER; +REVOKE ALL PRIVILEGES ON FUNCTION testagg3(int2) FROM "current_user"; +REVOKE ALL PRIVILEGES ON FUNCTION testagg4(int2) FROM SESSION_USER; +REVOKE ALL PRIVILEGES ON FUNCTION testagg5(int2) FROM "Public"; +REVOKE ALL PRIVILEGES ON FUNCTION testagg6(int2) FROM regress_testrolx; +REVOKE ALL PRIVILEGES ON FUNCTION testagg7(int2) FROM "public"; +REVOKE ALL PRIVILEGES ON FUNCTION testagg8(int2) + FROM current_user, public, regress_testrolx; + +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM CURRENT_ROLE; --error +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM USER; --error +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM NONE; --error +REVOKE ALL PRIVILEGES ON FUNCTION testagg9(int2) FROM "none"; --error + +SELECT proname, proacl FROM pg_proc WHERE proname LIKE 'testagg_'; + +-- clean up +\c + +DROP SCHEMA test_roles_schema; +DROP OWNED BY regress_testrol0, "Public", "current_user", regress_testrol1, regress_testrol2, regress_testrolx CASCADE; +DROP ROLE regress_testrol0, regress_testrol1, regress_testrol2, regress_testrolx; +DROP ROLE "Public", "None", "current_user", "session_user", "user"; diff --git a/src/postgres/src/test/regress/yb_pg_auth_serial_schedule b/src/postgres/src/test/regress/yb_pg_auth_serial_schedule new file mode 100644 index 000000000000..8d524d01a6f6 --- /dev/null +++ b/src/postgres/src/test/regress/yb_pg_auth_serial_schedule @@ -0,0 +1,11 @@ +# src/test/regress/yb_pg_auth_serial_schedule +# +#################################################################################################### +# Postgres Testsuites: Porting from PostgreSQL original tests. +# This includes tests related to postgres roles. +#################################################################################################### +test: yb_roleattributes +test: yb_rolenames +test: yb_password +test: yb_privileges +test: yb_init_privs diff --git a/src/yb/yql/pggate/pggate.cc b/src/yb/yql/pggate/pggate.cc index 64bf45f41bae..35cb61911b2e 100644 --- a/src/yb/yql/pggate/pggate.cc +++ b/src/yb/yql/pggate/pggate.cc @@ -434,7 +434,26 @@ Status PgApiImpl::GetColumnInfo(YBCPgTableDesc table_desc, return table_desc->GetColumnInfo(attr_number, is_primary, is_hash); } -Status PgApiImpl::SetIfIsSysCatalogVersionChange(PgStatement *handle, bool *is_version_change) { +Status PgApiImpl::DmlModifiesRow(PgStatement *handle, bool *modifies_row) { + if (!handle) { + return STATUS(InvalidArgument, "Invalid statement handle"); + } + + *modifies_row = false; + + switch (handle->stmt_op()) { + case StmtOp::STMT_UPDATE: + case StmtOp::STMT_DELETE: + *modifies_row = true; + break; + default: + break; + } + + return Status::OK(); +} + +Status PgApiImpl::SetIsSysCatalogVersionChange(PgStatement *handle) { if (!handle) { return STATUS(InvalidArgument, "Invalid statement handle"); } @@ -442,11 +461,8 @@ Status PgApiImpl::SetIfIsSysCatalogVersionChange(PgStatement *handle, bool *is_v switch (handle->stmt_op()) { case StmtOp::STMT_UPDATE: case StmtOp::STMT_DELETE: - *is_version_change = true; - down_cast(handle)->SetIsSystemCatalogChange(); - return Status::OK(); case StmtOp::STMT_INSERT: - *is_version_change = false; + down_cast(handle)->SetIsSystemCatalogChange(); return Status::OK(); default: break; diff --git a/src/yb/yql/pggate/pggate.h b/src/yb/yql/pggate/pggate.h index 9311dbf86ccb..e943ed3eb315 100644 --- a/src/yb/yql/pggate/pggate.h +++ b/src/yb/yql/pggate/pggate.h @@ -221,7 +221,9 @@ class PgApiImpl { bool *is_primary, bool *is_hash); - CHECKED_STATUS SetIfIsSysCatalogVersionChange(PgStatement *handle, bool *is_version_change); + CHECKED_STATUS DmlModifiesRow(PgStatement *handle, bool *modifies_row); + + CHECKED_STATUS SetIsSysCatalogVersionChange(PgStatement *handle); CHECKED_STATUS SetCatalogCacheVersion(PgStatement *handle, uint64_t catalog_cache_version); diff --git a/src/yb/yql/pggate/pggate_flags.cc b/src/yb/yql/pggate/pggate_flags.cc index e0e3b7dc156a..a9c3002717ab 100644 --- a/src/yb/yql/pggate/pggate_flags.cc +++ b/src/yb/yql/pggate/pggate_flags.cc @@ -66,5 +66,8 @@ DEFINE_bool(ysql_beta_feature_function, false, DEFINE_bool(ysql_beta_feature_trigger, false, "Whether to enable the 'trigger' ysql beta feature"); +DEFINE_bool(ysql_beta_feature_roles, false, + "Whether to enable the 'roles' ysql beta feature"); + DEFINE_bool(ysql_beta_feature_extension, false, "Whether to enable the 'extension' ysql beta feature"); diff --git a/src/yb/yql/pggate/pggate_flags.h b/src/yb/yql/pggate/pggate_flags.h index 591de231dd92..5c82be785caf 100644 --- a/src/yb/yql/pggate/pggate_flags.h +++ b/src/yb/yql/pggate/pggate_flags.h @@ -30,6 +30,7 @@ DECLARE_int32(ysql_session_max_batch_size); DECLARE_bool(ysql_beta_features); DECLARE_bool(ysql_beta_feature_function); DECLARE_bool(ysql_beta_feature_trigger); +DECLARE_bool(ysql_beta_feature_roles); DECLARE_bool(ysql_beta_feature_extension); #endif // YB_YQL_PGGATE_PGGATE_FLAGS_H diff --git a/src/yb/yql/pggate/ybc_pggate.cc b/src/yb/yql/pggate/ybc_pggate.cc index 6d6bfb34c5b4..5039c283758e 100644 --- a/src/yb/yql/pggate/ybc_pggate.cc +++ b/src/yb/yql/pggate/ybc_pggate.cc @@ -317,8 +317,12 @@ YBCStatus YBCPgSetCatalogCacheVersion(YBCPgStatement handle, return ToYBCStatus(pgapi->SetCatalogCacheVersion(handle, catalog_cache_version)); } -YBCStatus YBCPgSetIfIsSysCatalogVersionChange(YBCPgStatement handle, bool *is_version_change) { - return ToYBCStatus(pgapi->SetIfIsSysCatalogVersionChange(handle, is_version_change)); +YBCStatus YBCPgDmlModifiesRow(YBCPgStatement handle, bool *modifies_row) { + return ToYBCStatus(pgapi->DmlModifiesRow(handle, modifies_row)); +} + +YBCStatus YBCPgSetIsSysCatalogVersionChange(YBCPgStatement handle) { + return ToYBCStatus(pgapi->SetIsSysCatalogVersionChange(handle)); } YBCStatus YBCPgNewTruncateTable(YBCPgSession pg_session, diff --git a/src/yb/yql/pggate/ybc_pggate.h b/src/yb/yql/pggate/ybc_pggate.h index f9e9758a54c2..af94226b10f5 100644 --- a/src/yb/yql/pggate/ybc_pggate.h +++ b/src/yb/yql/pggate/ybc_pggate.h @@ -193,7 +193,9 @@ YBCStatus YBCPgGetColumnInfo(YBCPgTableDesc table_desc, bool *is_primary, bool *is_hash); -YBCStatus YBCPgSetIfIsSysCatalogVersionChange(YBCPgStatement handle, bool *is_version_change); +YBCStatus YBCPgDmlModifiesRow(YBCPgStatement handle, bool *modifies_row); + +YBCStatus YBCPgSetIsSysCatalogVersionChange(YBCPgStatement handle); YBCStatus YBCPgSetCatalogCacheVersion(YBCPgStatement handle, uint64_t catalog_cache_version); diff --git a/src/yb/yql/pgwrapper/pg_wrapper.cc b/src/yb/yql/pgwrapper/pg_wrapper.cc index 1e247f1b7e9c..66423c2d9cab 100644 --- a/src/yb/yql/pgwrapper/pg_wrapper.cc +++ b/src/yb/yql/pgwrapper/pg_wrapper.cc @@ -35,6 +35,7 @@ DEFINE_bool(pg_transactions_enabled, true, "True to enable transactions in YugaByte PostgreSQL API. This should eventually " "be set to true by default."); DEFINE_int32(pgsql_proxy_webserver_port, 13000, "Webserver port for PGSQL"); +DEFINE_test_flag(string, pgsql_hba_conf_file, "", "PGSQL host-based authentication file path"); DECLARE_string(metric_node_name); TAG_FLAG(pg_transactions_enabled, advanced); TAG_FLAG(pg_transactions_enabled, hidden); @@ -113,6 +114,11 @@ Status PgWrapper::Start() { argv.push_back("-c"); argv.push_back("yb_pg_metrics.port=" + std::to_string(FLAGS_pgsql_proxy_webserver_port)); + if (!FLAGS_pgsql_hba_conf_file.empty()) { + argv.push_back("-c"); + argv.push_back("hba_file=" + FLAGS_pgsql_hba_conf_file); + } + pg_proc_.emplace(postgres_executable, argv); pg_proc_->ShareParentStderr(); pg_proc_->ShareParentStdout(); @@ -143,21 +149,6 @@ Status PgWrapper::InitDb(bool yb_enabled) { exit_code); } - { - string hba_conf_path = JoinPathSegments(conf_.data_dir, "pg_hba.conf"); - std::ofstream hba_conf_file; - - hba_conf_file.open(hba_conf_path, std::ios_base::app); - hba_conf_file << std::endl; - hba_conf_file << "host all all 0.0.0.0/0 trust" << std::endl; - hba_conf_file << "host all all ::0/0 trust" << std::endl; - - if (!hba_conf_file) { - return STATUS(IOError, "Could not append additional lines to file " + hba_conf_path, - ErrnoToString(errno), errno); - } - } - LOG(INFO) << "initdb completed successfully. Database initialized at " << conf_.data_dir; return Status::OK(); }