You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "nuggetwheat (via GitHub)" <gi...@apache.org> on 2023/02/01 00:48:32 UTC

[GitHub] [beam] nuggetwheat opened a new pull request, #25246: Added Role-based access control integration tests for Spanner Change …

nuggetwheat opened a new pull request, #25246:
URL: https://github.com/apache/beam/pull/25246

   Expanded SpannerChangeStreamIT integration test to include role-based access control tests. The added tests runs with a separate metadata database to ensure this method of operation works properly.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "Abacn (via GitHub)" <gi...@apache.org>.
Abacn commented on PR #25246:
URL: https://github.com/apache/beam/pull/25246#issuecomment-1421250244

   Run PostCommit_Java_DataflowV2


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] hmehrotra commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "hmehrotra (via GitHub)" <gi...@apache.org>.
hmehrotra commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1093664151


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/ChangeStreamTestPipelineOptions.java:
##########
@@ -37,8 +37,20 @@ public interface ChangeStreamTestPipelineOptions extends IOTestPipelineOptions,
   void setInstanceId(String value);
 
   @Description("Database ID prefix to write to in Spanner")
-  @Default.String("changestream")
+  @Default.String("cs_primary")
   String getDatabaseId();
 
   void setDatabaseId(String value);
+
+  @Description("Role-Based Access Control Database ID prefix to write to in Spanner")
+  @Default.String("cs_rbac")
+  String getRbacDatabaseId();

Review Comment:
   Do we need a separate database for RBAC? Why can't we use the same DatabaseId? You would still need to create a separate DB client for rbac.



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java:
##########
@@ -48,18 +49,44 @@ public class IntegrationTestEnv extends ExternalResource {
   private static final String METADATA_TABLE_NAME_PREFIX = "TestMetadata";
   private static final String SINGERS_TABLE_NAME_PREFIX = "Singers";
   private static final String CHANGE_STREAM_NAME_PREFIX = "SingersStream";
+  private static final String DATABASE_ROLE = "test_role";
+  private List<String> databases;
   private List<String> changeStreams;
   private List<String> tables;
 
   private String projectId;
   private String instanceId;
   private String databaseId;
+  private String rbacDatabaseId;
+  private String metadataDatabaseId;
   private String metadataTableName;
   private Spanner spanner;
   private final String host = "https://spanner.googleapis.com";
   private DatabaseAdminClient databaseAdminClient;
   private DatabaseClient databaseClient;
+  private DatabaseClient rbacDatabaseClient;
   private boolean isPostgres;
+  public boolean createRbacDatabase;
+  public boolean useSeparateMetadataDb;
+
+  public IntegrationTestEnv withRbacDatabase() {
+    this.createRbacDatabase = true;

Review Comment:
   Use 'isRbac' as the parameter name to be consistent with 'isPostgres'?



##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java:
##########
@@ -70,154 +97,186 @@ protected void before() throws Throwable {
         Optional.ofNullable(options.getProjectId())
             .orElseGet(() -> options.as(GcpOptions.class).getProject());
     instanceId = options.getInstanceId();
-    databaseId = generateDatabaseName(options.getDatabaseId());
+    generateDatabaseIds(options);
     spanner =
         SpannerOptions.newBuilder().setProjectId(projectId).setHost(host).build().getService();
     databaseAdminClient = spanner.getDatabaseAdminClient();
     metadataTableName = generateTableName(METADATA_TABLE_NAME_PREFIX);
 
+    databases = new ArrayList<>();
     recreateDatabase(databaseAdminClient, instanceId, databaseId, isPostgres);
-
     databaseClient = spanner.getDatabaseClient(DatabaseId.of(projectId, instanceId, databaseId));
+    databases.add(databaseId);
+
+    if (createRbacDatabase) {
+      recreateDatabase(databaseAdminClient, instanceId, rbacDatabaseId, isPostgres);
+      rbacDatabaseClient =
+          spanner.getDatabaseClient(DatabaseId.of(projectId, instanceId, rbacDatabaseId));
+      databases.add(rbacDatabaseId);
+    }
+    if (useSeparateMetadataDb) {

Review Comment:
   Do we need a separate param for useSeparateMetadataDb? We could just check if (isRbac) here. Right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn merged pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "Abacn (via GitHub)" <gi...@apache.org>.
Abacn merged PR #25246:
URL: https://github.com/apache/beam/pull/25246


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] nuggetwheat commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "nuggetwheat (via GitHub)" <gi...@apache.org>.
nuggetwheat commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1095129143


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java:
##########
@@ -70,154 +97,186 @@ protected void before() throws Throwable {
         Optional.ofNullable(options.getProjectId())
             .orElseGet(() -> options.as(GcpOptions.class).getProject());
     instanceId = options.getInstanceId();
-    databaseId = generateDatabaseName(options.getDatabaseId());
+    generateDatabaseIds(options);
     spanner =
         SpannerOptions.newBuilder().setProjectId(projectId).setHost(host).build().getService();
     databaseAdminClient = spanner.getDatabaseAdminClient();
     metadataTableName = generateTableName(METADATA_TABLE_NAME_PREFIX);
 
+    databases = new ArrayList<>();
     recreateDatabase(databaseAdminClient, instanceId, databaseId, isPostgres);
-
     databaseClient = spanner.getDatabaseClient(DatabaseId.of(projectId, instanceId, databaseId));
+    databases.add(databaseId);
+
+    if (createRbacDatabase) {
+      recreateDatabase(databaseAdminClient, instanceId, rbacDatabaseId, isPostgres);
+      rbacDatabaseClient =
+          spanner.getDatabaseClient(DatabaseId.of(projectId, instanceId, rbacDatabaseId));
+      databases.add(rbacDatabaseId);
+    }
+    if (useSeparateMetadataDb) {

Review Comment:
   No longer relevant.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] nuggetwheat commented on pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "nuggetwheat (via GitHub)" <gi...@apache.org>.
nuggetwheat commented on PR #25246:
URL: https://github.com/apache/beam/pull/25246#issuecomment-1416329844

   Run Java_GCP_IO_Direct PreCommit
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "Abacn (via GitHub)" <gi...@apache.org>.
Abacn commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1095883554


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamIT.java:
##########
@@ -93,6 +101,22 @@ public void before() {
 
   @Test
   public void testReadSpannerChangeStream() {
+    testReadSpannerChangeStreamImpl(pipeline, null);
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithAuthorizedRole() {
+    testReadSpannerChangeStreamImpl(pipeline, ENV.getDatabaseRole());
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithUnauthorizedRole() {
+    exception.expect(SpannerException.class);

Review Comment:
   If we only need to validate the error handling on direct runner we can add something like `assumeTrue(options.getRunner() == DirectRunner.class);`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] nuggetwheat commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "nuggetwheat (via GitHub)" <gi...@apache.org>.
nuggetwheat commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1096334626


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamIT.java:
##########
@@ -93,6 +103,23 @@ public void before() {
 
   @Test
   public void testReadSpannerChangeStream() {
+    testReadSpannerChangeStreamImpl(pipeline, null);
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithAuthorizedRole() {

Review Comment:
   Yes. If someone (i.e. Apache Beam code) doesn't use the MetadataConfig constructor to generate the config to access the metadata DB in such a way that breaks access to the metadata DB, this test will catch it.  It is precisely why I modified this test to use a separate metadata database.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] nuggetwheat commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "nuggetwheat (via GitHub)" <gi...@apache.org>.
nuggetwheat commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1095128439


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java:
##########
@@ -48,18 +49,44 @@ public class IntegrationTestEnv extends ExternalResource {
   private static final String METADATA_TABLE_NAME_PREFIX = "TestMetadata";
   private static final String SINGERS_TABLE_NAME_PREFIX = "Singers";
   private static final String CHANGE_STREAM_NAME_PREFIX = "SingersStream";
+  private static final String DATABASE_ROLE = "test_role";
+  private List<String> databases;
   private List<String> changeStreams;
   private List<String> tables;
 
   private String projectId;
   private String instanceId;
   private String databaseId;
+  private String rbacDatabaseId;
+  private String metadataDatabaseId;
   private String metadataTableName;
   private Spanner spanner;
   private final String host = "https://spanner.googleapis.com";
   private DatabaseAdminClient databaseAdminClient;
   private DatabaseClient databaseClient;
+  private DatabaseClient rbacDatabaseClient;
   private boolean isPostgres;
+  public boolean createRbacDatabase;
+  public boolean useSeparateMetadataDb;
+
+  public IntegrationTestEnv withRbacDatabase() {
+    this.createRbacDatabase = true;

Review Comment:
   No longer relevant.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] github-actions[bot] commented on pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #25246:
URL: https://github.com/apache/beam/pull/25246#issuecomment-1421222052

   Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on PR #25246:
URL: https://github.com/apache/beam/pull/25246#issuecomment-1421220971

   John, can you review or ping the best person? (looks pretty easy but I want to respect boundaries of my expertise)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "Abacn (via GitHub)" <gi...@apache.org>.
Abacn commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1095874874


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamIT.java:
##########
@@ -93,6 +101,22 @@ public void before() {
 
   @Test
   public void testReadSpannerChangeStream() {
+    testReadSpannerChangeStreamImpl(pipeline, null);
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithAuthorizedRole() {
+    testReadSpannerChangeStreamImpl(pipeline, ENV.getDatabaseRole());
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithUnauthorizedRole() {
+    exception.expect(SpannerException.class);

Review Comment:
   the added tests will be executed on both direct runner precommit and dataflow postcommit:
   
   https://ci-beam.apache.org/view/PostCommit/job/beam_PreCommit_Java_GCP_IO_Direct_Cron
   
   https://ci-beam.apache.org/view/PostCommit/job/beam_PostCommit_Java_DataflowV2
   
   For the latter the exception would not thrown. If remote pipeline fails the PipelineResult is set to FAIL.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] nuggetwheat commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "nuggetwheat (via GitHub)" <gi...@apache.org>.
nuggetwheat commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1095128127


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/ChangeStreamTestPipelineOptions.java:
##########
@@ -37,8 +37,20 @@ public interface ChangeStreamTestPipelineOptions extends IOTestPipelineOptions,
   void setInstanceId(String value);
 
   @Description("Database ID prefix to write to in Spanner")
-  @Default.String("changestream")
+  @Default.String("cs_primary")
   String getDatabaseId();
 
   void setDatabaseId(String value);
+
+  @Description("Role-Based Access Control Database ID prefix to write to in Spanner")
+  @Default.String("cs_rbac")
+  String getRbacDatabaseId();

Review Comment:
   Done. We don't need a separate DB client for rbac, just need to specify SpannerConfig.databaseRole when in RBAC mode.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] nuggetwheat commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "nuggetwheat (via GitHub)" <gi...@apache.org>.
nuggetwheat commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1096058721


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamIT.java:
##########
@@ -93,6 +101,22 @@ public void before() {
 
   @Test
   public void testReadSpannerChangeStream() {
+    testReadSpannerChangeStreamImpl(pipeline, null);
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithAuthorizedRole() {
+    testReadSpannerChangeStreamImpl(pipeline, ENV.getDatabaseRole());
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithUnauthorizedRole() {
+    exception.expect(SpannerException.class);

Review Comment:
   Thanks for pointing this out! I've added the assumeTrue statement.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] hmehrotra commented on a diff in pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "hmehrotra (via GitHub)" <gi...@apache.org>.
hmehrotra commented on code in PR #25246:
URL: https://github.com/apache/beam/pull/25246#discussion_r1096273744


##########
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamIT.java:
##########
@@ -93,6 +103,23 @@ public void before() {
 
   @Test
   public void testReadSpannerChangeStream() {
+    testReadSpannerChangeStreamImpl(pipeline, null);
+  }
+
+  @Test
+  public void testReadSpannerChangeStreamWithAuthorizedRole() {

Review Comment:
   Have we tested that if someone breaks the pattern of not using MetadataConfig constructor, then this test fails?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on pull request #25246: Added Role-based access control integration tests for Spanner Change …

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on PR #25246:
URL: https://github.com/apache/beam/pull/25246#issuecomment-1421220394

   R: @johnjcasey


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org