def read_from_datastore(project, user_options, pipeline_options):
    """Creates a pipeline that reads entities from Cloud Datastore."""
    p = beam.Pipeline(options=pipeline_options)
    # Create a query to read entities from datastore.
    query = make_ancestor_query(user_options.kind, user_options.namespace,
                                user_options.ancestor)

    # Read entities from Cloud Datastore into a PCollection.
    lines = p | 'read from datastore' >> ReadFromDatastore(
        project, query, user_options.namespace)

    # Count the occurrences of each word.
    counts = (lines
              | 'split' >>
              (beam.ParDo(WordExtractingDoFn()).with_output_types(unicode))
              | 'pair_with_one' >> beam.Map(lambda x: (x, 1))
              | 'group' >> beam.GroupByKey()
              | 'count' >> beam.Map(lambda (word, ones): (word, sum(ones))))

    # Format the counts into a PCollection of strings.
    output = counts | 'format' >> beam.Map(lambda (word, c): '%s: %s' %
                                           (word, c))

    # Write the output using a "Write" transform that has side effects.
    # pylint: disable=expression-not-assigned
    output | 'write' >> beam.io.WriteToText(
        file_path_prefix=user_options.output,
        num_shards=user_options.num_shards)

    # Actually run the pipeline (all operations above are deferred).
    result = p.run()
    # Wait until completion, main thread would access post-completion job results.
    result.wait_until_finish()
    return result
Exemplo n.º 2
0
def model_datastoreio():
  """Using a Read and Write transform to read/write to Cloud Datastore."""

  import uuid
  from google.datastore.v1 import entity_pb2
  from google.datastore.v1 import query_pb2
  import googledatastore
  import apache_beam as beam
  from apache_beam.utils.pipeline_options import PipelineOptions
  from apache_beam.io.datastore.v1.datastoreio import ReadFromDatastore
  from apache_beam.io.datastore.v1.datastoreio import WriteToDatastore

  project = 'my_project'
  kind = 'my_kind'
  query = query_pb2.Query()
  query.kind.add().name = kind

  # [START model_datastoreio_read]
  p = beam.Pipeline(options=PipelineOptions())
  entities = p | 'Read From Datastore' >> ReadFromDatastore(project, query)
  # [END model_datastoreio_read]

  # [START model_datastoreio_write]
  p = beam.Pipeline(options=PipelineOptions())
  musicians = p | 'Musicians' >> beam.Create(
      ['Mozart', 'Chopin', 'Beethoven', 'Vivaldi'])

  def to_entity(content):
    entity = entity_pb2.Entity()
    googledatastore.helper.add_key_path(entity.key, kind, str(uuid.uuid4()))
    googledatastore.helper.add_properties(entity, {'content': unicode(content)})
    return entity

  entities = musicians | 'To Entity' >> beam.Map(to_entity)
  entities | 'Write To Datastore' >> WriteToDatastore(project)
Exemplo n.º 3
0
    def test_SplitQueryFn_without_num_splits(self):
        with patch.object(helper,
                          'get_datastore',
                          return_value=self._mock_datastore):
            # Force SplitQueryFn to compute the number of query splits
            num_splits = 0
            expected_num_splits = 23
            entity_bytes = (expected_num_splits *
                            ReadFromDatastore._DEFAULT_BUNDLE_SIZE_BYTES)
            with patch.object(ReadFromDatastore,
                              'get_estimated_size_bytes',
                              return_value=entity_bytes):

                def fake_get_splits(datastore,
                                    query,
                                    num_splits,
                                    partition=None):
                    return self.split_query(query, num_splits)

                with patch.object(query_splitter,
                                  'get_splits',
                                  side_effect=fake_get_splits):
                    split_query_fn = ReadFromDatastore.SplitQueryFn(
                        self._PROJECT, self._query, None, num_splits)
                    split_query_fn.start_bundle()
                    returned_split_queries = []
                    for split_query in split_query_fn.process(self._query):
                        returned_split_queries.append(split_query)

                    self.assertEqual(len(returned_split_queries),
                                     expected_num_splits)
                    self.assertEqual(
                        0, len(self._mock_datastore.run_query.call_args_list))
                    self.verify_unique_keys(returned_split_queries)
Exemplo n.º 4
0
    def test_SplitQueryFn_with_exception(self):
        """A test that verifies that no split is performed when failures occur."""
        with patch.object(helper,
                          'get_datastore',
                          return_value=self._mock_datastore):
            # Force SplitQueryFn to compute the number of query splits
            num_splits = 0
            expected_num_splits = 1
            entity_bytes = (expected_num_splits *
                            ReadFromDatastore._DEFAULT_BUNDLE_SIZE_BYTES)
            with patch.object(ReadFromDatastore,
                              'get_estimated_size_bytes',
                              return_value=entity_bytes):

                with patch.object(
                        query_splitter,
                        'get_splits',
                        side_effect=ValueError("Testing query split error")):
                    split_query_fn = ReadFromDatastore.SplitQueryFn(
                        self._PROJECT, self._query, None, num_splits)
                    split_query_fn.start_bundle()
                    returned_split_queries = []
                    for split_query in split_query_fn.process(self._query):
                        returned_split_queries.append(split_query)

                    self.assertEqual(len(returned_split_queries),
                                     expected_num_splits)
                    self.assertEqual(returned_split_queries[0][1], self._query)
                    self.assertEqual(
                        0, len(self._mock_datastore.run_query.call_args_list))
                    self.verify_unique_keys(returned_split_queries)
Exemplo n.º 5
0
  def check_estimated_size_bytes(self, entity_bytes, timestamp, namespace=None):
    """A helper method to test get_estimated_size_bytes"""

    timestamp_req = helper.make_request(
        self._PROJECT, namespace, helper.make_latest_timestamp_query(namespace))
    timestamp_resp = self.make_stats_response(
        {'timestamp': datastore_helper.from_timestamp(timestamp)})
    kind_stat_req = helper.make_request(
        self._PROJECT, namespace, helper.make_kind_stats_query(
            namespace, self._query.kind[0].name,
            datastore_helper.micros_from_timestamp(timestamp)))
    kind_stat_resp = self.make_stats_response(
        {'entity_bytes': entity_bytes})

    def fake_run_query(req):
      if req == timestamp_req:
        return timestamp_resp
      elif req == kind_stat_req:
        return kind_stat_resp
      else:
        print kind_stat_req
        raise ValueError("Unknown req: %s" % req)

    self._mock_datastore.run_query.side_effect = fake_run_query
    self.assertEqual(entity_bytes, ReadFromDatastore.get_estimated_size_bytes(
        self._PROJECT, namespace, self._query, self._mock_datastore))
    self.assertEqual(self._mock_datastore.run_query.call_args_list,
                     [call(timestamp_req), call(kind_stat_req)])
Exemplo n.º 6
0
    def test_SplitQueryFn_with_query_limit(self):
        """A test that verifies no split is performed when the query has a limit."""
        with patch.object(helper,
                          'get_datastore',
                          return_value=self._mock_datastore):
            self._query.limit.value = 3
            split_query_fn = ReadFromDatastore.SplitQueryFn(
                self._PROJECT, self._query, None, 4)
            split_query_fn.start_bundle()
            returned_split_queries = []
            for split_query in split_query_fn.process(self._query):
                returned_split_queries.append(split_query)

            self.assertEqual(1, len(returned_split_queries))
            self.assertEqual(0, len(self._mock_datastore.method_calls))
Exemplo n.º 7
0
  def test_SplitQueryFn_with_num_splits(self):
    with patch.object(helper, 'get_datastore',
                      return_value=self._mock_datastore):
      num_splits = 23

      def fake_get_splits(datastore, query, num_splits, partition=None):
        return self.split_query(query, num_splits)

      with patch.object(query_splitter, 'get_splits',
                        side_effect=fake_get_splits):

        split_query_fn = ReadFromDatastore.SplitQueryFn(
            self._PROJECT, self._query, None, num_splits)
        mock_context = MagicMock()
        mock_context.element = self._query
        split_query_fn.start_bundle(mock_context)
        returned_split_queries = []
        for split_query in split_query_fn.process(mock_context):
          returned_split_queries.append(split_query)

        self.assertEqual(len(returned_split_queries), num_splits)
        self.assertEqual(0, len(self._mock_datastore.run_query.call_args_list))
        self.verify_unique_keys(returned_split_queries)