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

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

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

 ##########
 File path: sdks/python/apache_beam/io/mongodbio.py
 ##########
 @@ -0,0 +1,358 @@
+#
+# 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 a configured
+MongoDB source and returns a ``PCollection`` of dict representing MongoDB
+documents.
+To configure MongoDB source, the URI to connect to MongoDB server, 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,
+               extra_client_params=None):
+    """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
+      extra_client_params(dict): Optional `MongoClient
+        
<https://api.mongodb.com/python/current/api/pymongo/mongo_client.html>`_
+        parameters
+
+    Returns:
+      :class:`~apache_beam.transforms.ptransform.PTransform`
+
+    """
+    if extra_client_params is None:
+      extra_client_params = {}
+    if not isinstance(db, str):
+      raise ValueError('ReadFromMongDB db param must be specified as a string')
+    if not isinstance(coll, str):
+      raise ValueError('ReadFromMongDB coll param must be specified as a '
+                       'string')
+    self._mongo_source = _BoundedMongoSource(
+        uri=uri,
+        db=db,
+        coll=coll,
+        filter=filter,
+        projection=projection,
+        extra_client_params=extra_client_params)
+
+  def expand(self, pcoll):
+    return pcoll | iobase.Read(self._mongo_source)
+
+
+class _BoundedMongoSource(iobase.BoundedSource):
+  def __init__(self,
+               uri=None,
+               db=None,
+               coll=None,
+               filter=None,
+               projection=None,
+               extra_client_params=None):
+    if extra_client_params is None:
+      extra_client_params = {}
+    if filter is None:
+      filter = {}
+    self.uri = uri
+    self.db = db
+    self.coll = coll
+    self.filter = filter
+    self.projection = projection
+    self.spec = extra_client_params
+    self.doc_count = self._get_document_count()
+    self.avg_doc_size = self._get_avg_document_size()
+    self.client = None
+
+  def estimate_size(self):
+    return self.avg_doc_size * self.doc_count
+
+  def split(self, desired_bundle_size, start_position=None, 
stop_position=None):
+    # use document cursor index as the start and stop positions
+    if start_position is None:
+      start_position = 0
+    if stop_position is None:
+      stop_position = self.doc_count
+
+    # get an estimate on how many documents should be included in a split batch
+    desired_bundle_count = desired_bundle_size // self.avg_doc_size
+
+    bundle_start = start_position
+    while bundle_start < stop_position:
+      bundle_end = min(stop_position, bundle_start + desired_bundle_count)
+      yield iobase.SourceBundle(weight=bundle_end - bundle_start,
+                                source=self,
+                                start_position=bundle_start,
+                                stop_position=bundle_end)
+      bundle_start = bundle_end
+
+  def get_range_tracker(self, start_position, stop_position):
+    if start_position is None:
+      start_position = 0
+    if stop_position is None:
+      stop_position = self.doc_count
+    return OffsetRangeTracker(start_position, stop_position)
+
+  def read(self, range_tracker):
+    with MongoClient(self.uri, **self.spec) as client:
+      # docs is a MongoDB Cursor
+      docs = client[self.db][self.coll].find(
+          filter=self.filter, projection=self.projection
+      )[range_tracker.start_position():range_tracker.stop_position()]
+      for index in range(range_tracker.start_position(),
+                         range_tracker.stop_position()):
+        if not range_tracker.try_claim(index):
+          return
+        yield docs[index - range_tracker.start_position()]
 
 Review comment:
   I guess this is what I intended here, the `docs` is a mongodb cursor that 
can iterate through documents between start_position and stop_position. And 
individual element is accessed through the offset. 
   for example if `docs` cover documents from position 30 to 60, then docs[0] 
represents the 30th document in mongodb, and docs[1] is the 31th.  
   Using yield docs[index] could be causing index error since the index here 
represents the overall position.
 
----------------------------------------------------------------
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: 261651)
    Time Spent: 6h 50m  (was: 6h 40m)

> 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: 6h 50m
>  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