[ 
https://issues.apache.org/jira/browse/BEAM-9072?focusedWorklogId=372630&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-372630
 ]

ASF GitHub Bot logged work on BEAM-9072:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 15/Jan/20 23:23
            Start Date: 15/Jan/20 23:23
    Worklog Time Spent: 10m 
      Work Description: apilloud commented on pull request #10440: [BEAM-9072] 
[SQL] DataStoreV1 IO connector
URL: https://github.com/apache/beam/pull/10440#discussion_r367158445
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreV1Table.java
 ##########
 @@ -0,0 +1,479 @@
+/*
+ * 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.extensions.sql.meta.provider.datastore;
+
+import static com.google.datastore.v1.Value.ValueTypeCase.ARRAY_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.BLOB_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.BOOLEAN_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.DOUBLE_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.ENTITY_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.INTEGER_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.KEY_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.NULL_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.STRING_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.TIMESTAMP_VALUE;
+import static com.google.datastore.v1.Value.ValueTypeCase.VALUETYPE_NOT_SET;
+import static com.google.datastore.v1.client.DatastoreHelper.makeKey;
+import static com.google.datastore.v1.client.DatastoreHelper.makeValue;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.alibaba.fastjson.JSONObject;
+import com.google.common.collect.ImmutableMap;
+import com.google.datastore.v1.Entity;
+import com.google.datastore.v1.Key;
+import com.google.datastore.v1.Query;
+import com.google.datastore.v1.Value;
+import com.google.datastore.v1.Value.ValueTypeCase;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.io.gcp.datastore.DatastoreIO;
+import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollection.IsBounded;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DataStoreV1Table extends SchemaBaseBeamTable implements 
Serializable {
+  public static final String KEY_FIELD_PROPERTY = "keyField";
+  @VisibleForTesting static final String DEFAULT_KEY_FIELD = "__key__";
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DataStoreV1Table.class);
+  // Should match: `projectId/kind`.
+  private static final Pattern locationPattern = 
Pattern.compile("(?<projectId>.+)/(?<kind>.+)");
+  @VisibleForTesting final String keyField;
+  @VisibleForTesting final String projectId;
+  @VisibleForTesting final String kind;
+
+  public DataStoreV1Table(Table table) {
+    super(table.getSchema());
+
+    // TODO: allow users to specify a name of the field to store a key value 
via TableProperties.
+    JSONObject properties = table.getProperties();
+    if (properties.containsKey(KEY_FIELD_PROPERTY)) {
+      String field = properties.getString(KEY_FIELD_PROPERTY);
+      checkArgument(
+          field != null && !field.isEmpty(), "'%s' property cannot be null.", 
KEY_FIELD_PROPERTY);
+      keyField = field;
+    } else {
+      keyField = DEFAULT_KEY_FIELD;
+    }
+    // TODO: allow users to specify a namespace in a location string.
+    String location = table.getLocation();
+    assert location != null;
+    Matcher matcher = locationPattern.matcher(location);
+    checkArgument(
+        matcher.matches(),
+        "DataStoreV1 location must be in the following format: 
'projectId/kind'");
+
+    this.projectId = matcher.group("projectId");
+    this.kind = matcher.group("kind");
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(PBegin begin) {
+    Query.Builder q = Query.newBuilder();
+    q.addKindBuilder().setName(kind);
+    Query query = q.build();
+
+    DatastoreV1.Read readInstance =
+        DatastoreIO.v1().read().withProjectId(projectId).withQuery(query);
+
+    PCollection<Entity> readEntities = readInstance.expand(begin);
+
+    return readEntities.apply(EntityToRow.create(getSchema(), keyField));
+  }
+
+  @Override
+  public POutput buildIOWriter(PCollection<Row> input) {
+    return input
+        .apply(RowToEntity.create(keyField, kind))
+        .apply(DatastoreIO.v1().write().withProjectId(projectId));
+  }
+
+  @Override
+  public IsBounded isBounded() {
+    return IsBounded.BOUNDED;
+  }
+
+  @Override
+  public BeamTableStatistics getTableStatistics(PipelineOptions options) {
+    long count =
+        
DatastoreIO.v1().read().withProjectId(projectId).getNumEntities(options, kind, 
null);
+
+    if (count < 0) {
+      return BeamTableStatistics.BOUNDED_UNKNOWN;
+    }
+
+    return BeamTableStatistics.createBoundedTableStatistics((double) count);
+  }
+
+  /**
+   * A {@code PTransform} to perform a conversion of {@code 
PCollection<Entity>} to {@code
+   * PCollection<Row>}.
+   */
+  public static class EntityToRow extends PTransform<PCollection<Entity>, 
PCollection<Row>> {
+    private final Schema schema;
+    private final String keyField;
+
+    private EntityToRow(Schema schema, String keyField) {
+      this.schema = schema;
+      this.keyField = keyField;
+
+      if (schema.getFieldNames().contains(keyField)
+          && 
!schema.getField(keyField).getType().getTypeName().equals(TypeName.BYTES)) {
+        throw new IllegalStateException(
+            "Field `"
+                + keyField
+                + "` should of type `VARBINARY`. Please change the type or 
specify a field to store the KEY value in via TableProperties.");
+      }
+    }
+
+    /**
+     * Create a PTransform instance.
+     *
+     * @param schema {@code Schema} of the target row.
+     * @return {@code PTransform} instance for Entity to Row conversion.
+     */
+    public static EntityToRow create(Schema schema) {
+      LOGGER.info(
+          "VARBINARY field to store KEY was not specified, using default 
value: `"
+              + DEFAULT_KEY_FIELD
+              + "`.");
+      return new EntityToRow(schema, DEFAULT_KEY_FIELD);
+    }
+
+    /**
+     * Create a PTransform instance.
+     *
+     * @param schema {@code Schema} of the target row.
+     * @param keyField A name of the row field to store the {@code Key} in.
+     * @return {@code PTransform} instance for Entity to Row conversion.
+     */
+    public static EntityToRow create(Schema schema, String keyField) {
+      LOGGER.info("VARBINARY field to store KEY was specified, using value: `" 
+ keyField + "`.");
 
 Review comment:
   This is probably too verbose and possibly confusing, as this will be printed 
even if the field wasn't specified.
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 372630)
    Time Spent: 1h 40m  (was: 1.5h)

> [SQL] Add support for Datastore source
> --------------------------------------
>
>                 Key: BEAM-9072
>                 URL: https://issues.apache.org/jira/browse/BEAM-9072
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Kirill Kozlov
>            Assignee: Kirill Kozlov
>            Priority: Major
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> * Create a Datastore table and table provider
>  * Conversion between Datastore and Beam data types
>  * Implement buildIOReader
>  * Implement buildIOWrite
>  * Implement getTableStatistics
> Doc: 
> [https://docs.google.com/document/d/1FxuEGewJ3GPDl0IKglfOYf1edwa2m_wryFZYRMpRNbA/edit?pli=1]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to