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

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

                Author: ASF GitHub Bot
            Created on: 13/Jun/19 03:06
            Start Date: 13/Jun/19 03:06
    Worklog Time Spent: 10m 
      Work Description: chamikaramj commented on pull request #8826: 
[BEAM-5148] Implement MongoDB IO for Python SDK
URL: https://github.com/apache/beam/pull/8826#discussion_r293187648
 
 

 ##########
 File path: sdks/python/apache_beam/io/mongodbio.py
 ##########
 @@ -0,0 +1,292 @@
+#
+# 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.
+#
+"""This module implements IO classes to read and write data on MongoDB.
+
+
+Read from MongoDB
+-----------------
+:class:`ReadFromMongoDB` is a ``PTransform`` that reads from configured MongoDB
+source and returns ``PCollection`` of dict representing MongoDB document.
+To configure MongoDB source, the URI, database name, collection name needs to 
be
+provided.
+
+Example usage::
+
+  pipeline | ReadFromMongoDB(uri='mongodb://localhost:27017',
+                             db='testdb',
+                             coll='input')
+
+
+Write to MongoDB:
+-----------------
+:class:`WriteToMongoDB` is a ``PTransform`` that writes MongoDB documents to
+configured sink, and the write is conducted through a mongodb bulk_write of
+``ReplaceOne`` operations. If the document's _id field already existed in the
+MongoDB collection, it results in an overwrite, otherwise, a new document
+will be inserted.
+
+Example usage::
+
+  pipeline | WriteToMongoDB(uri='mongodb://localhost:27017',
+                            db='testdb',
+                            coll='output',
+                            batch_size=10)
+
+
+No backward compatibility guarantees. Everything in this module is 
experimental.
+"""
+
+from __future__ import absolute_import
+
+from bson import objectid
+from pymongo import MongoClient
+from pymongo import ReplaceOne
+
+import apache_beam as beam
+from apache_beam.io import iobase
+from apache_beam.io.range_trackers import OffsetRangeTracker
+from apache_beam.transforms import DoFn
+from apache_beam.transforms import PTransform
+from apache_beam.transforms import Reshuffle
+from apache_beam.utils.annotations import experimental
+
+__all__ = ['ReadFromMongoDB', 'WriteToMongoDB']
+
+
+@experimental()
+class ReadFromMongoDB(PTransform):
+  """A ``PTransfrom`` to read MongoDB documents into a ``PCollection``.
+  """
+
+  def __init__(self,
+               uri='mongodb://localhost:27017',
+               db=None,
+               coll=None,
+               filter=None,
+               projection=None,
+               **kwargs):
+    """Initialize a :class:`ReadFromMongoDB`
+
+    Args:
+      uri (str): The MongoDB connection string following the URI format
+      db (str): The MongoDB database name
+      coll (str): The MongoDB collection name
+      filter: A `bson.SON
+        <https://api.mongodb.com/python/current/api/bson/son.html>`_ object
+        specifying elements which must be present for a document to be included
+        in the result set
+      projection: A list of field names that should be returned in the result
+        set or a dict specifying the fields to include or exclude
+      **kwargs: Optional `MongoClient
+        
<https://api.mongodb.com/python/current/api/pymongo/mongo_client.html>`_
+        parameters as keyword arguments
+
+    Returns:
+      :class:`~apache_beam.transforms.ptransform.PTransform`
+
+    """
+    self._mongo_source = _BoundedMongoSource(uri=uri,
 
 Review comment:
   Can we validate parameters passed to read and write transforms ?
 
----------------------------------------------------------------
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: 259230)
    Time Spent: 2h  (was: 1h 50m)

> Implement MongoDB IO for Python SDK
> -----------------------------------
>
>                 Key: BEAM-5148
>                 URL: https://issues.apache.org/jira/browse/BEAM-5148
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-py-core
>    Affects Versions: 3.0.0
>            Reporter: Pascal Gula
>            Assignee: Yichi Zhang
>            Priority: Major
>             Fix For: Not applicable
>
>          Time Spent: 2h
>  Remaining Estimate: 0h
>
> Currently Java SDK has MongoDB support but Python SDK does not. With current 
> portability efforts other runners may soon be able to use Python SDK. Having 
> mongoDB support will allow these runners to execute large scale jobs using it.
> Since we need this IO components @ Peat, we started working on a PyPi package 
> available at this repository: [https://github.com/PEAT-AI/beam-extended]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to