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

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

                Author: ASF GitHub Bot
            Created on: 25/Oct/19 23:59
            Start Date: 25/Oct/19 23:59
    Worklog Time Spent: 10m 
      Work Description: TheNeuralBit commented on pull request #9806: 
[BEAM-8427] Create a table and a table provider for MongoDB
URL: https://github.com/apache/beam/pull/9806#discussion_r339267118
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
 ##########
 @@ -0,0 +1,118 @@
+/*
+ * 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.mongodb;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.sdk.annotations.Experimental;
+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.mongodb.MongoDbIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.JsonToRow;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.ParDo.SingleOutput;
+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.bson.Document;
+import org.bson.json.JsonMode;
+import org.bson.json.JsonWriterSettings;
+
+@Experimental
+public class MongoDbTable extends SchemaBaseBeamTable implements Serializable {
+  // Should match: 
mongodb://username:password@localhost:27017/database/collection
+  @VisibleForTesting
+  final Pattern locationPattern =
+      Pattern.compile(
+          
"(?<credsHostPort>mongodb://(?<usernamePassword>.*:.*@)?.+:\\d+)/(?<database>.+)/(?<collection>.+)");
+
+  @VisibleForTesting final String dbCollection;
+  @VisibleForTesting final String dbName;
+  @VisibleForTesting final String dbUri;
+
+  MongoDbTable(Table table) {
+    super(table.getSchema());
+
+    String location = table.getLocation();
+    Matcher matcher = locationPattern.matcher(location);
+    checkArgument(
+        matcher.matches(),
+        "MongoDb location must be in the following format: 
'mongodb://(username:password@)?localhost:27017/database/collection'");
+    this.dbUri = matcher.group("credsHostPort"); // "mongodb://localhost:27017"
+    this.dbName = matcher.group("database");
+    this.dbCollection = matcher.group("collection");
+  }
+
+  @Override
+  public PCollection<Row> buildIOReader(PBegin begin) {
+    // Read MongoDb Documents
+    PCollection<Document> readDocuments =
+        MongoDbIO.read()
+            .withUri(dbUri)
+            .withDatabase(dbName)
+            .withCollection(dbCollection)
+            .expand(begin);
+
+    return readDocuments
+        // TODO: figure out a way convert Document directly to Row.
+        .apply("Convert Document to JSON", createParserParDo())
+        .apply("Transform JSON to Row", JsonToRow.withSchema(getSchema()))
+        .setRowSchema(getSchema());
+  }
+
+  @Override
+  public POutput buildIOWriter(PCollection<Row> input) {
+    throw new UnsupportedOperationException("Writing to a MongoDB is not 
supported");
+  }
+
+  @Override
+  public IsBounded isBounded() {
+    return IsBounded.BOUNDED;
+  }
+
+  @Override
+  public BeamTableStatistics getTableStatistics(PipelineOptions options) {
+    return BeamTableStatistics.BOUNDED_UNKNOWN;
 
 Review comment:
   Does MongoDB have any interface we could use to implement this? If so a 
TODO/Jira pointing in the right direction would be nice. If not, maybe add a 
comment stating why.
 
----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 334439)
    Time Spent: 40m  (was: 0.5h)

> [SQL] Add support for MongoDB source
> ------------------------------------
>
>                 Key: BEAM-8427
>                 URL: https://issues.apache.org/jira/browse/BEAM-8427
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Kirill Kozlov
>            Assignee: Kirill Kozlov
>            Priority: Major
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> In progress:
>  * Create a MongoDB table and table provider.
>  * Implement buildIOReader
>  * Support primitive types
> Still needs to be done:
>  * Implement buildIOWrite
>  * improve getTableStatistics



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

Reply via email to