Skip to content

Commit b474059

Browse files
authored
test(connector): add test cases for postgres validation permission checks (risingwavelabs#8662)
1 parent 4738ee9 commit b474059

File tree

2 files changed

+101
-5
lines changed

2 files changed

+101
-5
lines changed

java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/PostgresSourceTest.java

Lines changed: 67 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,12 +17,11 @@
1717
import static org.assertj.core.api.Assertions.assertThat;
1818
import static org.assertj.core.api.Assertions.fail;
1919
import static org.junit.Assert.assertEquals;
20+
import static org.junit.Assert.assertThrows;
2021

2122
import com.risingwave.proto.ConnectorServiceProto;
22-
import io.grpc.Grpc;
23-
import io.grpc.InsecureChannelCredentials;
24-
import io.grpc.Server;
25-
import io.grpc.ServerBuilder;
23+
import com.risingwave.proto.Data;
24+
import io.grpc.*;
2625
import java.io.IOException;
2726
import java.sql.Connection;
2827
import java.sql.ResultSet;
@@ -41,7 +40,7 @@ public class PostgresSourceTest {
4140
private static final Logger LOG = LoggerFactory.getLogger(PostgresSourceTest.class.getName());
4241

4342
private static final PostgreSQLContainer<?> pg =
44-
new PostgreSQLContainer<>("postgres:12.3-alpine")
43+
new PostgreSQLContainer<>("postgres:15-alpine")
4544
.withDatabaseName("test")
4645
.withUsername("postgres")
4746
.withCommand("postgres -c wal_level=logical -c max_wal_senders=10");
@@ -145,6 +144,69 @@ public void testLines() throws InterruptedException, SQLException {
145144
connection.close();
146145
}
147146

147+
// test whether validation catches permission errors
148+
@Test
149+
public void testPermissionCheck() {
150+
Connection connection = SourceTestClient.connect(pgDataSource);
151+
String query =
152+
"CREATE TABLE IF NOT EXISTS orders (o_key BIGINT NOT NULL, o_val INT, PRIMARY KEY (o_key))";
153+
SourceTestClient.performQuery(connection, query);
154+
// create a partial publication, check whether error is reported
155+
query = "CREATE PUBLICATION dbz_publication FOR TABLE orders (o_key)";
156+
SourceTestClient.performQuery(connection, query);
157+
ConnectorServiceProto.TableSchema tableSchema =
158+
ConnectorServiceProto.TableSchema.newBuilder()
159+
.addColumns(
160+
ConnectorServiceProto.TableSchema.Column.newBuilder()
161+
.setName("o_key")
162+
.setDataType(Data.DataType.TypeName.INT64)
163+
.build())
164+
.addColumns(
165+
ConnectorServiceProto.TableSchema.Column.newBuilder()
166+
.setName("o_val")
167+
.setDataType(Data.DataType.TypeName.INT32)
168+
.build())
169+
.addPkIndices(0)
170+
.build();
171+
Iterator<ConnectorServiceProto.GetEventStreamResponse> eventStream1 =
172+
testClient.getEventStreamValidate(
173+
pg,
174+
ConnectorServiceProto.SourceType.POSTGRES,
175+
tableSchema,
176+
"test",
177+
"orders");
178+
StatusRuntimeException exception1 =
179+
assertThrows(
180+
StatusRuntimeException.class,
181+
() -> {
182+
eventStream1.hasNext();
183+
});
184+
assertEquals(
185+
exception1.getMessage(),
186+
"INVALID_ARGUMENT: INTERNAL: The publication 'dbz_publication' does not cover all necessary columns in table orders");
187+
query = "DROP PUBLICATION dbz_publication";
188+
SourceTestClient.performQuery(connection, query);
189+
// revoke superuser and replication, check if reports error
190+
query = "ALTER USER " + pg.getUsername() + " nosuperuser noreplication";
191+
SourceTestClient.performQuery(connection, query);
192+
Iterator<ConnectorServiceProto.GetEventStreamResponse> eventStream2 =
193+
testClient.getEventStreamValidate(
194+
pg,
195+
ConnectorServiceProto.SourceType.POSTGRES,
196+
tableSchema,
197+
"test",
198+
"orders");
199+
StatusRuntimeException exception2 =
200+
assertThrows(
201+
StatusRuntimeException.class,
202+
() -> {
203+
eventStream2.hasNext();
204+
});
205+
assertEquals(
206+
exception2.getMessage(),
207+
"INVALID_ARGUMENT: INTERNAL: Postgres user must be superuser or replication role to start walsender.");
208+
}
209+
148210
// generates test cases for the risingwave debezium parser
149211
@Ignore
150212
@Test

java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/SourceTestClient.java

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,40 @@ protected static DataSource getDataSource(JdbcDatabaseContainer<?> container) {
8888
return new HikariDataSource(hikariConfig);
8989
}
9090

91+
protected Iterator<ConnectorServiceProto.GetEventStreamResponse> getEventStreamValidate(
92+
JdbcDatabaseContainer<?> container,
93+
ConnectorServiceProto.SourceType sourceType,
94+
ConnectorServiceProto.TableSchema tableSchema,
95+
String databaseName,
96+
String tableName) {
97+
String port = String.valueOf(URI.create(container.getJdbcUrl().substring(5)).getPort());
98+
ConnectorServiceProto.GetEventStreamRequest req =
99+
ConnectorServiceProto.GetEventStreamRequest.newBuilder()
100+
.setValidate(
101+
ConnectorServiceProto.GetEventStreamRequest.ValidateProperties
102+
.newBuilder()
103+
.setSourceId(0)
104+
.setSourceType(sourceType)
105+
.setTableSchema(tableSchema)
106+
.putProperties("hostname", container.getHost())
107+
.putProperties("port", port)
108+
.putProperties("username", container.getUsername())
109+
.putProperties("password", container.getPassword())
110+
.putProperties("database.name", databaseName)
111+
.putProperties("table.name", tableName)
112+
.putProperties("schema.name", "public") // pg only
113+
.putProperties("slot.name", "orders") // pg only
114+
.putProperties("server.id", "1")) // mysql only
115+
.build();
116+
Iterator<ConnectorServiceProto.GetEventStreamResponse> responses = null;
117+
try {
118+
responses = blockingStub.getEventStream(req);
119+
} catch (StatusRuntimeException e) {
120+
fail("RPC failed: {}", e.getStatus());
121+
}
122+
return responses;
123+
}
124+
91125
protected Iterator<ConnectorServiceProto.GetEventStreamResponse> getEventStreamStart(
92126
JdbcDatabaseContainer<?> container,
93127
ConnectorServiceProto.SourceType sourceType,

0 commit comments

Comments
 (0)