thiagotnunes commented on a change in pull request #16203:
URL: https://github.com/apache/beam/pull/16203#discussion_r780038736



##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dao/PartitionMetadataDao.java
##########
@@ -0,0 +1,470 @@
+/*
+ * 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.dao;
+
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.ChangeStreamMetrics.PARTITION_ID_ATTRIBUTE_LABEL;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_CREATED_AT;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_END_TIMESTAMP;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_FINISHED_AT;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_HEARTBEAT_MILLIS;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_PARENT_TOKENS;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_PARTITION_TOKEN;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_RUNNING_AT;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_SCHEDULED_AT;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_START_TIMESTAMP;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_STATE;
+import static 
org.apache.beam.sdk.io.gcp.spanner.changestreams.dao.PartitionMetadataAdminDao.COLUMN_WATERMARK;
+
+import com.google.cloud.Timestamp;
+import com.google.cloud.spanner.DatabaseClient;
+import com.google.cloud.spanner.Mutation;
+import com.google.cloud.spanner.ResultSet;
+import com.google.cloud.spanner.Statement;
+import com.google.cloud.spanner.Struct;
+import com.google.cloud.spanner.TransactionContext;
+import com.google.cloud.spanner.TransactionRunner;
+import com.google.cloud.spanner.Value;
+import io.opencensus.common.Scope;
+import io.opencensus.trace.AttributeValue;
+import io.opencensus.trace.Tracer;
+import io.opencensus.trace.Tracing;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+import javax.annotation.Nullable;
+import 
org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionMetadata;
+import 
org.apache.beam.sdk.io.gcp.spanner.changestreams.model.PartitionMetadata.State;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+/** Data access object for the Connector metadata tables. */
+public class PartitionMetadataDao {
+  private static final Tracer TRACER = Tracing.getTracer();
+
+  private final String metadataTableName;
+  private final DatabaseClient databaseClient;
+
+  /**
+   * Constructs a partition metadata dao object given the generated name of 
the tables.
+   *
+   * @param metadataTableName the name of the partition metadata table
+   * @param databaseClient the {@link DatabaseClient} to perform queries
+   */
+  PartitionMetadataDao(String metadataTableName, DatabaseClient 
databaseClient) {
+    this.metadataTableName = metadataTableName;
+    this.databaseClient = databaseClient;
+  }
+
+  /**
+   * Fetches the partition metadata row data for the given partition token.
+   *
+   * @param partitionToken the partition unique identifier
+   * @return the partition metadata for the given token if it exists as a 
struct. Otherwise, it
+   *     returns null.
+   */
+  public @Nullable Struct getPartition(String partitionToken) {
+    try (Scope scope = 
TRACER.spanBuilder("getPartition").setRecordEvents(true).startScopedSpan()) {
+      TRACER
+          .getCurrentSpan()
+          .putAttribute(
+              PARTITION_ID_ATTRIBUTE_LABEL, 
AttributeValue.stringAttributeValue(partitionToken));
+      try (ResultSet resultSet =
+          databaseClient
+              .singleUse()
+              .executeQuery(
+                  Statement.newBuilder(
+                          "SELECT * FROM "
+                              + metadataTableName
+                              + " WHERE "
+                              + COLUMN_PARTITION_TOKEN
+                              + " = @partition")
+                      .bind("partition")
+                      .to(partitionToken)
+                      .build())) {
+        if (resultSet.next()) {
+          return resultSet.getCurrentRowAsStruct();
+        }
+        return null;

Review comment:
       It should not happen. If this happens we throw an 
`IllegalStateException` on the caller.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to