Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[WIP] try new protobuf #30556

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
27 changes: 14 additions & 13 deletions sdks/python/apache_beam/transforms/window.py
Original file line number Diff line number Diff line change
Expand Up @@ -277,7 +277,7 @@ def intersects(self, other):
def union(self, other):
# type: (IntervalWindow) -> IntervalWindow
return IntervalWindow(
min(self.start, other.start), max(self.end, other.end))
start=min(self.start, other.start), end=max(self.end, other.end))


V = TypeVar("V")
Expand Down Expand Up @@ -447,7 +447,7 @@ def assign(self, context):
# type: (WindowFn.AssignContext) -> List[IntervalWindow]
timestamp = context.timestamp
start = timestamp - (timestamp - self.offset) % self.size
return [IntervalWindow(start, start + self.size)]
return [IntervalWindow(start=start, end=start + self.size)]

def get_window_coder(self):
# type: () -> coders.IntervalWindowCoder
Expand Down Expand Up @@ -476,8 +476,8 @@ def to_runner_api_parameter(self, context):
def from_runner_api_parameter(fn_parameter, unused_context):
# type: (...) -> FixedWindows
return FixedWindows(
size=Duration(micros=fn_parameter.size.ToMicroseconds()),
offset=Timestamp(micros=fn_parameter.offset.ToMicroseconds()))
size=Duration(micros=proto_utils.to_micros(fn_parameter.size)),
offset=Timestamp(micros=proto_utils.to_micros(fn_parameter.offset)))


class SlidingWindows(NonMergingWindowFn):
Expand Down Expand Up @@ -512,8 +512,8 @@ def assign(self, context):
start = timestamp - ((timestamp - self.offset) % self.period)
return [
IntervalWindow(
(interval_start := Timestamp(micros=s)),
interval_start + self.size,
start=(interval_start := Timestamp(micros=s)),
end=interval_start + self.size,
) for s in range(
start.micros,
timestamp.micros - self.size.micros,
Expand Down Expand Up @@ -551,9 +551,9 @@ def to_runner_api_parameter(self, context):
def from_runner_api_parameter(fn_parameter, unused_context):
# type: (...) -> SlidingWindows
return SlidingWindows(
size=Duration(micros=fn_parameter.size.ToMicroseconds()),
offset=Timestamp(micros=fn_parameter.offset.ToMicroseconds()),
period=Duration(micros=fn_parameter.period.ToMicroseconds()))
size=Duration(micros=proto_utils.to_micros(fn_parameter.size)),
offset=Timestamp(micros=proto_utils.to_micros(fn_parameter.offset)),
period=Duration(micros=proto_utils.to_micros(fn_parameter.period)))


class Sessions(WindowFn):
Expand All @@ -574,7 +574,7 @@ def __init__(self, gap_size):
def assign(self, context):
# type: (WindowFn.AssignContext) -> List[IntervalWindow]
timestamp = context.timestamp
return [IntervalWindow(timestamp, timestamp + self.gap_size)]
return [IntervalWindow(start=timestamp, end=timestamp + self.gap_size)]

def get_window_coder(self):
# type: () -> coders.IntervalWindowCoder
Expand All @@ -593,14 +593,15 @@ def merge(self, merge_context):
else:
if len(to_merge) > 1:
merge_context.merge(
to_merge, IntervalWindow(to_merge[0].start, end))
to_merge, IntervalWindow(start=to_merge[0].start, end=end))
to_merge = [w]
end = w.end
else:
to_merge = [w]
end = w.end
if len(to_merge) > 1:
merge_context.merge(to_merge, IntervalWindow(to_merge[0].start, end))
merge_context.merge(
to_merge, IntervalWindow(start=to_merge[0].start, end=end))

def __eq__(self, other):
if type(self) == type(other) == Sessions:
Expand All @@ -623,4 +624,4 @@ def to_runner_api_parameter(self, context):
def from_runner_api_parameter(fn_parameter, unused_context):
# type: (...) -> Sessions
return Sessions(
gap_size=Duration(micros=fn_parameter.gap_size.ToMicroseconds()))
gap_size=Duration(micros=proto_utils.to_micros(fn_parameter.gap_size)))
26 changes: 23 additions & 3 deletions sdks/python/apache_beam/utils/proto_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
#

"""For internal use only; no backwards-compatibility guarantees."""

# pytype: skip-file

from typing import Type
Expand All @@ -36,6 +35,10 @@

message_types = (message.Message, )

_MICROS_TO_SECONDS = 1000000
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
_MICROS_TO_SECONDS = 1000000
_SECONDS_TO_MICROS = 10**6

ditto elsewhere

_MICRO_TO_NANOSECONDS = 1000
_NANO_TO_SECONDS = 1000000000


@overload
def pack_Any(msg):
Expand Down Expand Up @@ -123,8 +126,25 @@ def pack_Struct(**kwargs):
def from_micros(cls, micros):
# type: (Type[TimeMessageT], int) -> TimeMessageT
result = cls()
result.FromMicroseconds(micros)
return result
if isinstance(result, duration_pb2.Duration):
result.FromMicroseconds(micros)
return result
elif isinstance(result, timestamp_pb2.Timestamp):
result.seconds = micros // _MICROS_TO_SECONDS
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we add some comments here about why we do these conversions?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. add brief comment why we don't use timestamp_pb2.Timestamp.FromMicroseconds()
and a reference an issue that has details.

result.nanos = (micros % _MICROS_TO_SECONDS) * _MICRO_TO_NANOSECONDS
return result
else:
raise RuntimeError('cannot convert the micro seconds to %s' % cls)


def to_micros(value: Union[duration_pb2.Duration, timestamp_pb2.Timestamp]):
if isinstance(value, duration_pb2.Duration):
return value.ToMicroseconds()
elif isinstance(value, timestamp_pb2.Timestamp):
micros = value.seconds * _MICROS_TO_SECONDS
return micros + value.nanos // _MICRO_TO_NANOSECONDS
else:
raise RuntimeError('cannot convert %s to micro seconds' % value)


def to_Timestamp(time):
Expand Down
41 changes: 41 additions & 0 deletions sdks/python/apache_beam/utils/proto_utils_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
#
# 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.
#
import unittest

from google.protobuf import duration_pb2
from google.protobuf import timestamp_pb2

from apache_beam.utils import proto_utils
from apache_beam.utils.timestamp import MAX_TIMESTAMP


class TestProtoUtils(unittest.TestCase):
def test_from_micros_duration(self):
ts = proto_utils.from_micros(duration_pb2.Duration, MAX_TIMESTAMP.micros)
expected = duration_pb2.Duration(
seconds=MAX_TIMESTAMP.seconds(), nanos=775000000)
self.assertEqual(ts, expected)

def test_from_micros_timestamp(self):
ts = proto_utils.from_micros(timestamp_pb2.Timestamp, MAX_TIMESTAMP.micros)
expected = timestamp_pb2.Timestamp(
seconds=MAX_TIMESTAMP.seconds(), nanos=775000000)
self.assertEqual(ts, expected)


if __name__ == '__main__':
unittest.main()
Loading