diff --git a/sdks/java/extensions/google-cloud-platform-core/OWNERS b/sdks/java/extensions/google-cloud-platform-core/OWNERS
index da65659085da..7f25f229e967 100644
--- a/sdks/java/extensions/google-cloud-platform-core/OWNERS
+++ b/sdks/java/extensions/google-cloud-platform-core/OWNERS
@@ -3,3 +3,6 @@
reviewers:
- lukecwik
- chamikaramj
+ - pabloem
+ - ihji
+ - johnjcasey
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java
index a547bc9e8c01..3e540f5731fa 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java
@@ -58,7 +58,16 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-/** {@link FileSystem} implementation for Google Cloud Storage. */
+/**
+ * {@link FileSystem} implementation for Google Cloud Storage.
+ *
+ *
Updates to the I/O connector code
+ *
+ * For any significant updates to this I/O connector, please consider involving corresponding code
+ * reviewers mentioned
+ * here.
+ */
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
diff --git a/sdks/java/io/google-cloud-platform/OWNERS b/sdks/java/io/google-cloud-platform/OWNERS
index da65659085da..cfe68754c999 100644
--- a/sdks/java/io/google-cloud-platform/OWNERS
+++ b/sdks/java/io/google-cloud-platform/OWNERS
@@ -1,5 +1,36 @@
# See the OWNERS docs at https://s.apache.org/beam-owners
-reviewers:
+reviewers (BigTableIO) :
+ - igorbernstein2
+
+reviewers (SpannerIO) :
+ - darshan-sj
+ - nielm
+
+reviewers (SpannerIO - readChangeStream) :
+ - cl2012
+ - nancyxu123
+ - haikuo-google
+
+reviewers (BigQueryIO - Storage Read API) :
+ - kmjung
+ - vachan-shetty
+
+reviewers (FhirIO) :
+ - msbukal
+
+reviewers (DatastoreIO) :
+ - pcostell
+
+reviewers (FirestoreIO) :
+ - pcostell
+
+reviewers (PubsubLiteIO) :
+ - dpcollins-google
+
+reviewers (all GCP connectors):
- lukecwik
- chamikaramj
+ - pabloem
+ - ihji
+ - johnjcasey
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 2a3e595adc1f..6596d33be099 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -457,6 +457,13 @@
*
* Please see BigQuery Access Control
* for security and permission related information specific to BigQuery.
+ *
+ *
Updates to the I/O connector code
+ *
+ * For any significant updates to this I/O connector, please consider involving corresponding code
+ * reviewers mentioned
+ * here.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index 8fbadf218f95..36797092293c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -171,6 +171,13 @@
* Permission requirements depend on the {@link PipelineRunner} that is used to execute the
* pipeline. Please refer to the documentation of corresponding {@link PipelineRunner
* PipelineRunners} for more details.
+ *
+ *
Updates to the I/O connector code
+ *
+ * For any significant updates to this I/O connector, please consider involving corresponding code
+ * reviewers mentioned
+ * here.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index 7e88de712b69..d540e7379ca2 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -210,6 +210,11 @@
* above transforms. In such a case, all the Cloud Datastore API calls are directed to the Emulator.
*
* @see PipelineRunner
+ * Updates to the connector code
+ * For any updates to this connector, please consider involving corresponding code reviewers
+ * mentioned
+ * here.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java
index d5facd34cef3..ee2bfba5ed4c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java
@@ -233,6 +233,13 @@
* listSearchResult.getKeyedResources(); // KV<"Alice-Bob-Search", JsonArray of results>
*
*
+ *
+ * Updates to the I/O connector code
+ *
+ * For any significant updates to this I/O connector, please consider involving corresponding code
+ * reviewers mentioned
+ *
+ * here.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
index 484fe4f77f8e..7a276a172b25 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
@@ -96,6 +96,13 @@
* Permission requirements depend on the {@link PipelineRunner} that is used to execute the Beam
* pipeline. Please refer to the documentation of corresponding {@link PipelineRunner
* PipelineRunners} for more details.
+ *
+ *
Updates to the I/O connector code
+ *
+ * For any significant updates to this I/O connector, please consider involving corresponding code
+ * reviewers mentioned
+ * here.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java
index b2cd1e67cc39..d484445e569d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java
@@ -35,6 +35,13 @@
*
* For the differences between this and Google Pub/Sub, please refer to the product documentation.
+ *
+ *
Updates to the I/O connector code
+ *
+ * For any significant updates to this I/O connector, please consider involving corresponding code
+ * reviewers mentioned
+ * here.
*/
@Experimental
public final class PubsubLiteIO {
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
index dee1c8020a75..9523ca6bbb89 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
@@ -353,6 +353,13 @@
*
* {@link Write} can be used as a streaming sink, however as with batch mode note that the write
* order of individual {@link Mutation}/{@link MutationGroup} objects is not guaranteed.
+ *
+ *
Updates to the I/O connector code
+ *
+ * For any significant significant updates to this I/O connector, please consider involving
+ * corresponding code reviewers mentioned
+ * here.
*/
@Experimental(Kind.SOURCE_SINK)
@SuppressWarnings({
diff --git a/sdks/java/io/kafka/OWNERS b/sdks/java/io/kafka/OWNERS
new file mode 100644
index 000000000000..6d38a74d6ccb
--- /dev/null
+++ b/sdks/java/io/kafka/OWNERS
@@ -0,0 +1,7 @@
+# See the OWNERS docs at https://s.apache.org/beam-owners
+
+reviewers:
+ - aromanenko-dev
+ - chamikaramj
+ - lukecwik
+ - johnjcasey
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
index a88df313ff9f..34ea6ef51b9e 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
@@ -533,6 +533,12 @@
* ensure that the version included with the application is compatible with the version of your
* Kafka cluster. Kafka client usually fails to initialize with a clear error message in case of
* incompatibility.
+ *
+ * Updates to the I/O connector code
+ *
+ * For any significant significant updates to this I/O connector, please consider involving
+ * corresponding code reviewers mentioned here.
*/
@Experimental(Kind.SOURCE_SINK)
@SuppressWarnings({
diff --git a/sdks/python/OWNERS b/sdks/python/OWNERS
index 7dce27abe126..164680516a64 100644
--- a/sdks/python/OWNERS
+++ b/sdks/python/OWNERS
@@ -1,5 +1,12 @@
# See the OWNERS docs at https://s.apache.org/beam-owners
-reviewers:
+reviewers (all changes):
- aaltay
- - charlesccychen
+ - tvalentyn
+
+reviewers (all GCP I/O connectors):
+ - lukecwik
+ - chamikaramj
+ - pabloem
+ - ihji
+ - johnjcasey
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index c7bc5560bbc2..2c21dca60478 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -265,6 +265,12 @@ def compute_table_name(row):
to BigQuery.
BigQuery IO requires values of BYTES datatype to be encoded using base64
encoding when writing to BigQuery.
+
+**Updates to the I/O connector code**
+
+For any significant updates to this I/O connector, please consider involving
+corresponding code reviewers mentioned in
+https://github.com/apache/beam/blob/master/sdks/python/OWNERS
"""
# pytype: skip-file
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
index dea51a0ce896..be912ae33761 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
@@ -20,6 +20,12 @@
This module uses the newer google-cloud-datastore client package. Its API was
different enough to require extensive changes to this and associated modules.
+
+**Updates to the I/O connector code**
+
+For any significant updates to this I/O connector, please consider involving
+corresponding code reviewers mentioned in
+https://github.com/apache/beam/blob/master/sdks/python/OWNERS
"""
# pytype: skip-file
diff --git a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py
index 6e4b1b6c7f0e..90ecc1d0fcc9 100644
--- a/sdks/python/apache_beam/io/gcp/gcsfilesystem.py
+++ b/sdks/python/apache_beam/io/gcp/gcsfilesystem.py
@@ -15,7 +15,14 @@
# limitations under the License.
#
-"""GCS file system implementation for accessing files on GCS."""
+"""GCS file system implementation for accessing files on GCS.
+
+**Updates to the I/O connector code**
+
+For any significant updates to this I/O connector, please consider involving
+corresponding code reviewers mentioned in
+https://github.com/apache/beam/blob/master/sdks/python/OWNERS
+"""
# pytype: skip-file
diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py
index 87654bc81743..599861b5f778 100644
--- a/sdks/python/apache_beam/io/gcp/gcsio.py
+++ b/sdks/python/apache_beam/io/gcp/gcsio.py
@@ -19,6 +19,12 @@
This library evolved from the Google App Engine GCS client available at
https://github.com/GoogleCloudPlatform/appengine-gcs-client.
+
+**Updates to the I/O connector code**
+
+For any significant updates to this I/O connector, please consider involving
+corresponding code reviewers mentioned in
+https://github.com/apache/beam/blob/master/sdks/python/OWNERS
"""
# pytype: skip-file
diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py
index 40d8c8f419f6..c3ff0ff3bc3b 100644
--- a/sdks/python/apache_beam/io/gcp/pubsub.py
+++ b/sdks/python/apache_beam/io/gcp/pubsub.py
@@ -21,6 +21,12 @@
pipelines, during remote execution.
This API is currently under development and is subject to change.
+
+**Updates to the I/O connector code**
+
+For any significant updates to this I/O connector, please consider involving
+corresponding code reviewers mentioned in
+https://github.com/apache/beam/blob/master/sdks/python/OWNERS
"""
# pytype: skip-file