WithTimestamps

Assigns timestamps to all the elements of a collection.

Examples

In the following examples, we create a pipeline with a PCollection and attach a timestamp value to each of its elements. When windowing and late data play an important role in streaming pipelines, timestamps are especially useful.

Example 1: Timestamp by event time

The elements themselves often already contain a timestamp field. beam.window.TimestampedValue takes a value and a Unix timestamp in the form of seconds.

import apache_beam as beam

class GetTimestamp(beam.DoFn):
  def process(self, plant, timestamp=beam.DoFn.TimestampParam):
    yield '{} - {}'.format(timestamp.to_utc_datetime(), plant['name'])

with beam.Pipeline() as pipeline:
  plant_timestamps = (
      pipeline
      | 'Garden plants' >> beam.Create([
          {'name': 'Strawberry', 'season': 1585699200}, # April, 2020
          {'name': 'Carrot', 'season': 1590969600},     # June, 2020
          {'name': 'Artichoke', 'season': 1583020800},  # March, 2020
          {'name': 'Tomato', 'season': 1588291200},     # May, 2020
          {'name': 'Potato', 'season': 1598918400},     # September, 2020
      ])
      | 'With timestamps' >> beam.Map(
          lambda plant: beam.window.TimestampedValue(plant, plant['season']))
      | 'Get timestamp' >> beam.ParDo(GetTimestamp())
      | beam.Map(print)
  )

Output PCollection after getting the timestamps:

2020-04-01 00:00:00 - Strawberry
2020-06-01 00:00:00 - Carrot
2020-03-01 00:00:00 - Artichoke
2020-05-01 00:00:00 - Tomato
2020-09-01 00:00:00 - Potato
Run code now Run code now
View source code View source code




To convert from a time.struct_time to unix_time you can use time.mktime. For more information on time formatting options, see time.strftime.

import time

time_tuple = time.strptime('2020-03-19 20:50:00', '%Y-%m-%d %H:%M:%S')
unix_time = time.mktime(time_tuple)

To convert from a datetime.datetime to unix_time you can use convert it to a time.struct_time first with datetime.timetuple.

import time
import datetime

now = datetime.datetime.now()
time_tuple = now.timetuple()
unix_time = time.mktime(time_tuple)

Example 2: Timestamp by logical clock

If each element has a chronological number, these numbers can be used as a logical clock. These numbers have to be converted to a “seconds” equivalent, which can be especially important depending on your windowing and late data rules.

import apache_beam as beam

class GetTimestamp(beam.DoFn):
  def process(self, plant, timestamp=beam.DoFn.TimestampParam):
    event_id = int(timestamp.micros / 1e6)  # equivalent to seconds
    yield '{} - {}'.format(event_id, plant['name'])

with beam.Pipeline() as pipeline:
  plant_events = (
      pipeline
      | 'Garden plants' >> beam.Create([
          {'name': 'Strawberry', 'event_id': 1},
          {'name': 'Carrot', 'event_id': 4},
          {'name': 'Artichoke', 'event_id': 2},
          {'name': 'Tomato', 'event_id': 3},
          {'name': 'Potato', 'event_id': 5},
      ])
      | 'With timestamps' >> beam.Map(lambda plant: \
          beam.window.TimestampedValue(plant, plant['event_id']))
      | 'Get timestamp' >> beam.ParDo(GetTimestamp())
      | beam.Map(print)
  )

Output PCollection after getting the timestamps:

1 - Strawberry
4 - Carrot
2 - Artichoke
3 - Tomato
5 - Potato
Run code now Run code now
View source code View source code




Example 3: Timestamp by processing time

If the elements do not have any time data available, you can also use the current processing time for each element. Note that this grabs the local time of the worker that is processing each element. Workers might have time deltas, so using this method is not a reliable way to do precise ordering.

By using processing time, there is no way of knowing if data is arriving late because the timestamp is attached when the element enters into the pipeline.

import apache_beam as beam
import time

class GetTimestamp(beam.DoFn):
  def process(self, plant, timestamp=beam.DoFn.TimestampParam):
    yield '{} - {}'.format(timestamp.to_utc_datetime(), plant['name'])

with beam.Pipeline() as pipeline:
  plant_processing_times = (
      pipeline
      | 'Garden plants' >> beam.Create([
          {'name': 'Strawberry'},
          {'name': 'Carrot'},
          {'name': 'Artichoke'},
          {'name': 'Tomato'},
          {'name': 'Potato'},
      ])
      | 'With timestamps' >> beam.Map(lambda plant: \
          beam.window.TimestampedValue(plant, time.time()))
      | 'Get timestamp' >> beam.ParDo(GetTimestamp())
      | beam.Map(print)
  )

Output PCollection after getting the timestamps:

2020-03-20 20:12:42.145594 - Strawberry
2020-03-20 20:12:42.145827 - Carrot
2020-03-20 20:12:42.145962 - Artichoke
2020-03-20 20:12:42.146093 - Tomato
2020-03-20 20:12:42.146216 - Potato
Run code now Run code now
View source code View source code