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/01/26 19:36:14 UTC

[GitHub] [beam] nuggetwheat opened a new pull request, #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   This is a follow-up change to the quick fix that was implemented in https://github.com/apache/beam/pull/25108.  It is a more robust solution that makes the code less brittle so that a similar config issue doesn't crop up again in the future.  For details of the problem and why this solution is necessary, see https://github.com/apache/beam/pull/25108#issuecomment-1400709831.
   


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {
+
+  /**
+   * Generates an SpannerConfig that can be used to access the change stream metadata database by
+   * copying only the necessary fields from the given primary database SpannerConfig and setting the
+   * instance ID and database ID to the supplied metadata values.
+   *
+   * @param primaryConfig The SpannerConfig for accessing the primary database
+   * @param metadataInstanceId The instance ID of the metadata database
+   * @param metadataDatabaseId The database ID of the metadata database
+   * @return the metadata SpannerConfig
+   */
+  public static SpannerConfig create(
+      SpannerConfig primaryConfig, String metadataInstanceId, String metadataDatabaseId) {
+
+    checkNotNull(
+        metadataInstanceId, "MetadataSpannerConfig.create requires non-null metadata instance id");
+    checkNotNull(
+        metadataDatabaseId, "MetadataSpannerConfig.create requires non-null metadata database id");
+
+    // NOTE: databaseRole should NOT be copied to the metadata config
+
+    SpannerConfig config =

Review Comment:
   Is there anything prevent us from just copying a SpannerConfig instance but assign them one by one except for databaseRole?
   
   If proceed with this approach, probably consider adding a unit test to check the consistency between the input and output config, in case of more fields added in the future but forgot to be added here.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {
+
+  /**
+   * Generates an SpannerConfig that can be used to access the change stream metadata database by

Review Comment:
   ```suggestion
      * Generates a SpannerConfig that can be used to access the change stream metadata database by
   ```



-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   Reminder, please take a look at this pr: @lukecwik @pabloem 


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {
+
+  /**
+   * Generates an SpannerConfig that can be used to access the change stream metadata database by
+   * copying only the necessary fields from the given primary database SpannerConfig and setting the
+   * instance ID and database ID to the supplied metadata values.
+   *
+   * @param primaryConfig The SpannerConfig for accessing the primary database
+   * @param metadataInstanceId The instance ID of the metadata database
+   * @param metadataDatabaseId The database ID of the metadata database
+   * @return the metadata SpannerConfig
+   */
+  public static SpannerConfig create(
+      SpannerConfig primaryConfig, String metadataInstanceId, String metadataDatabaseId) {
+
+    checkNotNull(
+        metadataInstanceId, "MetadataSpannerConfig.create requires non-null metadata instance id");
+    checkNotNull(
+        metadataDatabaseId, "MetadataSpannerConfig.create requires non-null metadata database id");
+
+    // NOTE: databaseRole should NOT be copied to the metadata config
+
+    SpannerConfig config =

Review Comment:
   If a field gets added to SpannerConfig in the future, it's unknown if it should be copied to the Metadata config or not, so coming up with a unit test now to verify the correct output of this factory with a future added field to SpannerConfig isn't possible.
   
   https://github.com/apache/beam/pull/25246 introduces an integration test that uses a metadata database that is distinct from the primary database. If a new field gets added to SpannerConfig that only makes sense for the primary database, then this factory will not copy it into the Metadata config and the code will work properly with no change. If the new field should get copied to the Metadata config, then the integration test will fail and will require a change to this factory class. Likewise, if someone introduces code that copies the SpannerConfig and only strips databaseRole and does not handle the new field properly, the integration test will catch it.



-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   looks like jenkins not triggering. will try again


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {
+
+  /**
+   * Generates an SpannerConfig that can be used to access the change stream metadata database by

Review Comment:
   Done.



##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {

Review Comment:
   Done.



-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {
+
+  /**
+   * Generates an SpannerConfig that can be used to access the change stream metadata database by
+   * copying only the necessary fields from the given primary database SpannerConfig and setting the
+   * instance ID and database ID to the supplied metadata values.
+   *
+   * @param primaryConfig The SpannerConfig for accessing the primary database
+   * @param metadataInstanceId The instance ID of the metadata database
+   * @param metadataDatabaseId The database ID of the metadata database
+   * @return the metadata SpannerConfig
+   */
+  public static SpannerConfig create(
+      SpannerConfig primaryConfig, String metadataInstanceId, String metadataDatabaseId) {
+
+    checkNotNull(
+        metadataInstanceId, "MetadataSpannerConfig.create requires non-null metadata instance id");
+    checkNotNull(
+        metadataDatabaseId, "MetadataSpannerConfig.create requires non-null metadata database id");
+
+    // NOTE: databaseRole should NOT be copied to the metadata config
+
+    SpannerConfig config =

Review Comment:
   thanks for the detailed explanation. sounds good to me



-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   test failure Java_Examples_Dataflow_Java17 unrelated. merging for now. Thanks!


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   retest this please


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   please avoid rebase unless merge conflict. this mixes review history. tests already run on feature branch merged onto master


-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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


##########
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfig.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.apache.beam.sdk.io.gcp.spanner.changestreams;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.api.gax.retrying.RetrySettings;
+import com.google.api.gax.rpc.StatusCode.Code;
+import com.google.cloud.spanner.Options.RpcPriority;
+import org.apache.beam.sdk.io.gcp.spanner.SpannerConfig;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.joda.time.Duration;
+
+/**
+ * This class generates a SpannerConfig for the change stream metadata database by copying only the
+ * necessary fields from the SpannerConfig of the primary database.
+ */
+public class MetadataSpannerConfig {

Review Comment:
   MetadataSpannerConfigFactory?



-- 
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 #25193: Added MetadataSpannerConfig class for generating SpannerConfig for ac…

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

   created #25383 for tracking. this PR would close it.


-- 
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