From 409a31a388e660599bd9a93a6c6361cac6d1d556 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 22 Nov 2024 10:13:14 -0800 Subject: [PATCH 01/33] Add bounded Trie metric type. --- sdks/python/apache_beam/metrics/cells.pxd | 11 + sdks/python/apache_beam/metrics/cells.py | 207 +++++++++++++++++ sdks/python/apache_beam/metrics/cells_test.py | 218 ++++++++++++++++++ 3 files changed, 436 insertions(+) diff --git a/sdks/python/apache_beam/metrics/cells.pxd b/sdks/python/apache_beam/metrics/cells.pxd index c583dabeb0c0..7590bd8b5966 100644 --- a/sdks/python/apache_beam/metrics/cells.pxd +++ b/sdks/python/apache_beam/metrics/cells.pxd @@ -60,3 +60,14 @@ cdef class DistributionData(object): cdef readonly libc.stdint.int64_t count cdef readonly libc.stdint.int64_t min cdef readonly libc.stdint.int64_t max + + +cdef class _BoundedTrieNode(object): + cdef readonly libc.stdint.int64_t _size + cdef readonly dict _children + cdef readonly bint _truncated + +cdef class BoundedTrieData(object): + cdef readonly libc.stdint.int64_t _bound + cdef readonly object _singleton + cdef readonly _BoundedTrieNode _root diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py index 10ac7b3a1e69..f04468b00b5f 100644 --- a/sdks/python/apache_beam/metrics/cells.py +++ b/sdks/python/apache_beam/metrics/cells.py @@ -23,6 +23,7 @@ # pytype: skip-file +import copy import logging import threading import time @@ -312,6 +313,35 @@ def to_runner_api_monitoring_info_impl(self, name, transform_id): ptransform=transform_id) +class BoundedTrieCell(AbstractMetricCell): + """For internal use only; no backwards-compatibility guarantees. + + Tracks the current value for a StringSet metric. + + Each cell tracks the state of a metric independently per context per bundle. + Therefore, each metric has a different cell in each bundle, that is later + aggregated. + + This class is thread safe. + """ + def __init__(self): + super().__init__(BoundedTrieData) + + def add(self, value): + self.update(value) + + def _update_locked(self, value): + self.data.add(value) + + def to_runner_api_monitoring_info_impl(self, name, transform_id): + from apache_beam.metrics import monitoring_infos + return monitoring_infos.user_bounded_trie( + name.namespace, + name.name, + self.get_cumulative(), + ptransform=transform_id) + + class DistributionResult(object): """The result of a Distribution metric.""" def __init__(self, data): @@ -630,3 +660,180 @@ def singleton(value: str) -> "StringSetData": @staticmethod def identity_element() -> "StringSetData": return StringSetData() + + +class _BoundedTrieNode(object): + def __init__(self): + # invariant: size = len(self.flattened()) = min(1, sum(size of children)) + self._size = 1 + self._children: Optional[dict[str, '_BoundedTrieNode']] = {} + self._truncated = False + + def size(self): + return self._size + + def add(self, segments) -> int: + if self._truncated or not segments: + return 0 + head, *tail = segments + was_empty = not self._children + child = self._children.get(head, None) # type: ignore[union-attr] + if child is None: + child = self._children[head] = _BoundedTrieNode() # type: ignore[index] + delta = 0 if was_empty else 1 + else: + delta = 0 + if tail: + delta += child.add(tail) + self._size += delta + return delta + + def add_all(self, segments_iter): + return sum(self.add(segments) for segments in segments_iter) + + def trim(self) -> int: + if not self._children: + return 0 + max_child = max(self._children.values(), key=lambda child: child._size) + if max_child._size == 1: + delta = 1 - self._size + self._truncated = True + self._children = None + else: + delta = max_child.trim() + self._size += delta + return delta + + def merge(self, other: '_BoundedTrieNode') -> int: + if self._truncated: + delta = 0 + elif other._truncated: + delta = 1 - self._size + self._truncated = True + self._children = None + elif not other._children: + delta = 0 + elif not self._children: + self._children = other._children + delta = self._size - other._size + else: + delta = 0 + other_child: '_BoundedTrieNode' + self_child: Optional['_BoundedTrieNode'] + for prefix, other_child in other._children.items(): + self_child = self._children.get(prefix, None) + if self_child is None: + self._children[prefix] = other_child + delta += other_child._size + else: + delta += self_child.merge(other_child) + self._size += delta + return delta + + def flattened(self): + if self._truncated: + yield (True, ) + elif not self._children: + yield (False, ) + else: + for prefix, child in sorted(self._children.items()): + for flattened in child.flattened(): + yield (prefix, ) + flattened + + def __hash__(self): + return self._truncated or hash(sorted(self._children.items())) + + def __eq__(self, other): + if isinstance(other, _BoundedTrieNode): + return ( + self._truncated == other._truncated and + self._children == other._children) + else: + return False + + def __repr__(self): + return repr(set(''.join(str(s) for s in t) for t in self.flattened())) + + +class BoundedTrieData(object): + _DEFAULT_BOUND = 100 + + def __init__(self, *, root=None, singleton=None, bound=_DEFAULT_BOUND): + assert singleton is None or root is None + self._singleton = singleton + self._root = root + self._bound = bound + + def as_trie(self): + if self._root is not None: + return self._root + else: + root = _BoundedTrieNode() + if self._singleton is not None: + root.add(self._singleton) + return root + + def __eq__(self, other: object) -> bool: + if isinstance(other, BoundedTrieData): + return self.as_trie() == other.as_trie() + else: + return False + + def __hash__(self) -> int: + return hash(self.as_trie()) + + def __repr__(self) -> str: + return 'BoundedTrieData({})'.format(self.as_trie()) + + def get_cumulative(self) -> "BoundedTrieData": + return copy.deepcopy(self) + + def get_result(self) -> set[tuple]: + if self._root is None: + if self._singleton is None: + return set() + else: + return set([self._singleton + (False, )]) + else: + return set(self._root.flattened()) + + def add(self, segments): + if self._root is None and self._singleton is None: + self._singleton = segments + elif self._singleton is not None and self._singleton == segments: + # Optimize for the common case of re-adding the same value. + return + else: + if self._root is None: + self._root = self.as_trie() + self._root.add(segments) + if self._root._size > self._bound: + self._root.trim() + + def combine(self, other: "BoundedTrieData") -> "BoundedTrieData": + if self._root is None and self._singleton is None: + return other + elif other._root is None and other._singleton is None: + return self + else: + if self._root is None and other._root is not None: + self, other = other, self + combined = copy.deepcopy(self.as_trie()) + if other._root is not None: + combined.merge(other._root) + else: + combined.add(other._singleton) + self._bound = min(self._bound, other._bound) + while combined._size > self._bound: + combined.trim() + return BoundedTrieData(root=combined) + + @staticmethod + def singleton(value: str) -> "BoundedTrieData": + s = BoundedTrieData() + s.add(value) + return s + + @staticmethod + def identity_element() -> "BoundedTrieData": + return BoundedTrieData() diff --git a/sdks/python/apache_beam/metrics/cells_test.py b/sdks/python/apache_beam/metrics/cells_test.py index d1ee37b8ed82..68e3f12a73fd 100644 --- a/sdks/python/apache_beam/metrics/cells_test.py +++ b/sdks/python/apache_beam/metrics/cells_test.py @@ -17,9 +17,13 @@ # pytype: skip-file +import copy +import itertools +import random import threading import unittest +from apache_beam.metrics.cells import BoundedTrieData from apache_beam.metrics.cells import CounterCell from apache_beam.metrics.cells import DistributionCell from apache_beam.metrics.cells import DistributionData @@ -27,6 +31,7 @@ from apache_beam.metrics.cells import GaugeData from apache_beam.metrics.cells import StringSetCell from apache_beam.metrics.cells import StringSetData +from apache_beam.metrics.cells import _BoundedTrieNode from apache_beam.metrics.metricbase import MetricName @@ -203,5 +208,218 @@ def test_add_size_tracked_correctly(self): self.assertEqual(s.data.string_size, 3) +class TestBoundedTrieNode(unittest.TestCase): + @classmethod + def random_segments_fixed_depth(cls, n, depth, overlap, rand): + if depth == 0: + yield from ((), ) * n + else: + seen = [] + to_string = lambda ix: chr(ord('a') + ix) if ix < 26 else f'z{ix}' + for suffix in cls.random_segments_fixed_depth(n, depth - 1, overlap, + rand): + if not seen or rand.random() > overlap: + prefix = to_string(len(seen)) + seen.append(prefix) + else: + prefix = rand.choice(seen) + yield (prefix, ) + suffix + + @classmethod + def random_segments(cls, n, min_depth, max_depth, overlap, rand): + for depth, segments in zip( + itertools.cycle(range(min_depth, max_depth + 1)), + cls.random_segments_fixed_depth(n, max_depth, overlap, rand)): + yield segments[:depth] + + def assert_covers(self, node, expected, max_truncated=0): + self.assert_covers_flattened(node.flattened(), expected, max_truncated) + + def assert_covers_flattened(self, flattened, expected, max_truncated=0): + expected = set(expected) + # Split node into the exact and truncated segments. + partitioned = {True: set(), False: set()} + for segments in flattened: + partitioned[segments[-1]].add(segments[:-1]) + exact, truncated = partitioned[False], partitioned[True] + # Check we cover both parts. + self.assertLessEqual(len(truncated), max_truncated, truncated) + self.assertTrue(exact.issubset(expected), exact - expected) + seen_truncated = set() + for segments in expected - exact: + found = 0 + for ix in range(len(segments)): + if segments[:ix] in truncated: + seen_truncated.add(segments[:ix]) + found += 1 + if found != 1: + self.fail( + f"Expected exactly one prefix of {segments} " + f"to occur in {truncated}, found {found}") + self.assertEqual(seen_truncated, truncated, truncated - seen_truncated) + + def run_covers_test(self, flattened, expected, max_truncated): + def parse(s): + return tuple(s.strip('*')) + (s.endswith('*'), ) + + self.assert_covers_flattened([parse(s) for s in flattened], + [tuple(s) for s in expected], + max_truncated) + + def test_covers_exact(self): + self.run_covers_test(['ab', 'ac', 'cd'], ['ab', 'ac', 'cd'], 0) + with self.assertRaises(AssertionError): + self.run_covers_test(['ab', 'ac', 'cd'], ['ac', 'cd'], 0) + with self.assertRaises(AssertionError): + self.run_covers_test(['ab', 'ac'], ['ab', 'ac', 'cd'], 0) + with self.assertRaises(AssertionError): + self.run_covers_test(['a*', 'cd'], ['ab', 'ac', 'cd'], 0) + + def test_covers_trunacted(self): + self.run_covers_test(['a*', 'cd'], ['ab', 'ac', 'cd'], 1) + self.run_covers_test(['a*', 'cd'], ['ab', 'ac', 'abcde', 'cd'], 1) + with self.assertRaises(AssertionError): + self.run_covers_test(['ab', 'ac', 'cd'], ['ac', 'cd'], 1) + with self.assertRaises(AssertionError): + self.run_covers_test(['ab', 'ac'], ['ab', 'ac', 'cd'], 1) + with self.assertRaises(AssertionError): + self.run_covers_test(['a*', 'c*'], ['ab', 'ac', 'cd'], 1) + with self.assertRaises(AssertionError): + self.run_covers_test(['a*', 'c*'], ['ab', 'ac'], 1) + + def run_test(self, to_add): + everything = list(set(to_add)) + all_prefixees = set( + segments[:ix] for segments in everything for ix in range(len(segments))) + everything_deduped = set(everything) - all_prefixees + + # Check basic addition. + node = _BoundedTrieNode() + total_size = node.size() + self.assertEqual(total_size, 1) + for segments in everything: + total_size += node.add(segments) + self.assertEqual(node.size(), len(everything_deduped), node) + self.assertEqual(node.size(), total_size, node) + self.assert_covers(node, everything_deduped) + + # Check merging + node0 = _BoundedTrieNode() + node0.add_all(everything[0::2]) + node1 = _BoundedTrieNode() + node1.add_all(everything[1::2]) + pre_merge_size = node0.size() + merge_delta = node0.merge(node1) + self.assertEqual(node0.size(), pre_merge_size + merge_delta) + self.assertEqual(node0, node) + + # Check trimming. + if node.size() > 1: + trim_delta = node.trim() + self.assertLess(trim_delta, 0, node) + self.assertEqual(node.size(), total_size + trim_delta) + self.assert_covers(node, everything_deduped, max_truncated=1) + + if node.size() > 1: + trim2_delta = node.trim() + self.assertLess(trim2_delta, 0) + self.assertEqual(node.size(), total_size + trim_delta + trim2_delta) + self.assert_covers(node, everything_deduped, max_truncated=2) + + # Adding after trimming should be a no-op. + node_copy = copy.deepcopy(node) + for segments in everything: + self.assertEqual(node.add(segments), 0) + self.assertEqual(node, node_copy) + + # Merging after trimming should be a no-op. + self.assertEqual(node.merge(node0), 0) + self.assertEqual(node.merge(node1), 0) + self.assertEqual(node, node_copy) + + if node._truncated: + expected_delta = 0 + else: + expected_delta = 2 + + # Adding something new is not. + new_values = [('new1', ), ('new2', 'new2.1')] + self.assertEqual(node.add_all(new_values), expected_delta) + self.assert_covers( + node, list(everything_deduped) + new_values, max_truncated=2) + + # Nor is merging something new. + new_values_node = _BoundedTrieNode() + new_values_node.add_all(new_values) + self.assertEqual(node_copy.merge(new_values_node), expected_delta) + self.assert_covers( + node_copy, list(everything_deduped) + new_values, max_truncated=2) + + def run_fuzz(self, iterations=10, **params): + for _ in range(iterations): + seed = random.getrandbits(64) + segments = self.random_segments(**params, rand=random.Random(seed)) + try: + self.run_test(segments) + except: + print("SEED", seed) + raise + + def test_trivial(self): + self.run_test([('a', 'b'), ('a', 'c')]) + + def test_flat(self): + self.run_test([('a', 'a'), ('b', 'b'), ('c', 'c')]) + + def test_deep(self): + self.run_test([('a', ) * 10, ('b', ) * 12]) + + def test_small(self): + self.run_fuzz(n=5, min_depth=2, max_depth=3, overlap=0.5) + + def test_medium(self): + self.run_fuzz(n=20, min_depth=2, max_depth=4, overlap=0.5) + + def test_large_sparse(self): + self.run_fuzz(n=120, min_depth=2, max_depth=4, overlap=0.2) + + def test_large_dense(self): + self.run_fuzz(n=120, min_depth=2, max_depth=4, overlap=0.8) + + def test_bounded_trie_data_combine(self): + empty = BoundedTrieData() + # The merging here isn't complicated we're just ensuring that + # BoundedTrieData invokes _BoundedTrieNode correctly. + singletonA = BoundedTrieData(singleton=('a', 'a')) + singletonB = BoundedTrieData(singleton=('b', 'b')) + lots_root = _BoundedTrieNode() + lots_root.add_all([('c', 'c'), ('d', 'd')]) + lots = BoundedTrieData(root=lots_root) + self.assertEqual(empty.get_result(), set()) + self.assertEqual( + empty.combine(singletonA).get_result(), set([('a', 'a', False)])) + self.assertEqual( + singletonA.combine(empty).get_result(), set([('a', 'a', False)])) + self.assertEqual( + singletonA.combine(singletonB).get_result(), + set([('a', 'a', False), ('b', 'b', False)])) + self.assertEqual( + singletonA.combine(lots).get_result(), + set([('a', 'a', False), ('c', 'c', False), ('d', 'd', False)])) + self.assertEqual( + lots.combine(singletonA).get_result(), + set([('a', 'a', False), ('c', 'c', False), ('d', 'd', False)])) + + def test_bounded_trie_data_combine_trim(self): + left = _BoundedTrieNode() + left.add_all([('a', 'x'), ('b', 'd')]) + right = _BoundedTrieNode() + right.add_all([('a', 'y'), ('c', 'd')]) + self.assertEqual( + BoundedTrieData(root=left).combine( + BoundedTrieData(root=right, bound=3)).get_result(), + set([('a', True), ('b', 'd', False), ('c', 'd', False)])) + + if __name__ == '__main__': unittest.main() From f83c6836ba0625dcdd86882c277edc0811fb0367 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 25 Nov 2024 11:55:15 -0800 Subject: [PATCH 02/33] Bounded Trie proto representation. --- .../beam/model/pipeline/v1/metrics.proto | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/metrics.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/metrics.proto index 4ec189e4637f..33bb5ae729f8 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/metrics.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/metrics.proto @@ -198,6 +198,17 @@ message MonitoringInfoSpecs { }] }]; + // Represents a set of strings seen across bundles. + USER_BOUNDED_TRIE = 22 [(monitoring_info_spec) = { + urn: "beam:metric:user:bounded_trie:v1", + type: "beam:metrics:bounded_trie:v1", + required_labels: ["PTRANSFORM", "NAMESPACE", "NAME"], + annotations: [{ + key: "description", + value: "URN utilized to report user metric." + }] + }]; + // General monitored state information which contains structured information // which does not fit into a typical metric format. See MonitoringTableData // for more details. @@ -576,6 +587,12 @@ message MonitoringInfoTypeUrns { SET_STRING_TYPE = 11 [(org.apache.beam.model.pipeline.v1.beam_urn) = "beam:metrics:set_string:v1"]; + // Represents a bounded trie of strings. + // + // Encoding: BoundedTrie proto + BOUNDED_TRIE_TYPE = 12 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:metrics:bounded_trie:v1"]; + // General monitored state information which contains structured information // which does not fit into a typical metric format. See MonitoringTableData // for more details. @@ -588,6 +605,30 @@ message MonitoringInfoTypeUrns { } } + +// A single node in a BoundedTrie. +message BoundedTrieNode { + // Whether this node has been truncated. + // A truncated leaf represents possibly many children with the same prefix. + bool truncated = 1; + + // Children of this node. Must be empty if truncated is true. + map children = 2; +} + +// The message type used for encoding metrics of type bounded trie. +message BoundedTrie { + // The maximum number of elements to store before truncation. + int32 bound = 1; + + // A compact representation of all the elements in this trie. + BoundedTrieNode root = 2; + + // A more efficient representation for metrics consisting of a single value. + repeated string singleton = 3; +} + + // General monitored state information which contains structured information // which does not fit into a typical metric format. // From d8358cb95b07d290a4e2b50ef1e9ddbb128fad98 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 25 Nov 2024 17:53:13 -0800 Subject: [PATCH 03/33] Bounded Trie translation. --- sdks/python/apache_beam/metrics/cells.py | 38 +++++++++++++++++++ .../apache_beam/metrics/monitoring_infos.py | 24 +++++++++++- 2 files changed, 61 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py index f04468b00b5f..e044693a6e9a 100644 --- a/sdks/python/apache_beam/metrics/cells.py +++ b/sdks/python/apache_beam/metrics/cells.py @@ -32,6 +32,8 @@ from typing import Optional from typing import Set +from apache_beam.portability.api import metrics_pb2 + try: import cython except ImportError: @@ -669,6 +671,29 @@ def __init__(self): self._children: Optional[dict[str, '_BoundedTrieNode']] = {} self._truncated = False + def to_proto(self) -> metrics_pb2.BoundedTrieNode: + return metrics_pb2.BoundedTrieNode( + truncated=self._truncated, + children={ + name: child.to_proto() + for name, child in self._children.items() + } if self._children else None) + + @staticmethod + def from_proto(proto: metrics_pb2.BoundedTrieNode) -> '_BoundedTrieNode': + node = _BoundedTrieNode() + if proto.truncated: + node._truncated = True + node._children = None + else: + node._children = { + name: _BoundedTrieNode.from_proto(child) + for name, + child in proto.children.items() + } + node._size = min(1, sum(child._size for child in node._children.values())) + return node + def size(self): return self._size @@ -764,6 +789,19 @@ def __init__(self, *, root=None, singleton=None, bound=_DEFAULT_BOUND): self._root = root self._bound = bound + def to_proto(self) -> metrics_pb2.BoundedTrie: + return metrics_pb2.BoundedTrie( + bound=self._bound, + singleton=self._singlton if self._singleton else None, + root=self._root.to_proto() if self._root else None) + + @staticmethod + def from_proto(proto: metrics_pb2.BoundedTrie) -> 'BoundedTrieData': + return BoundedTrieData( + bound=proto.bound, + singleton=tuple(proto.singleton) if proto.singleton else None, + root=_BoundedTrieNode.from_proto(proto.root) if proto.root else None) + def as_trie(self): if self._root is not None: return self._root diff --git a/sdks/python/apache_beam/metrics/monitoring_infos.py b/sdks/python/apache_beam/metrics/monitoring_infos.py index 5227a4c9872b..397fcc578d53 100644 --- a/sdks/python/apache_beam/metrics/monitoring_infos.py +++ b/sdks/python/apache_beam/metrics/monitoring_infos.py @@ -50,11 +50,14 @@ common_urns.monitoring_info_specs.USER_DISTRIBUTION_INT64.spec.urn) USER_GAUGE_URN = common_urns.monitoring_info_specs.USER_LATEST_INT64.spec.urn USER_STRING_SET_URN = common_urns.monitoring_info_specs.USER_SET_STRING.spec.urn +USER_BOUNDED_TRIE_URN = ( + common_urns.monitoring_info_specs.USER_BOUNDED_TRIE.spec.urn) USER_METRIC_URNS = set([ USER_COUNTER_URN, USER_DISTRIBUTION_URN, USER_GAUGE_URN, - USER_STRING_SET_URN + USER_STRING_SET_URN, + USER_BOUNDED_TRIE_URN, ]) WORK_REMAINING_URN = common_urns.monitoring_info_specs.WORK_REMAINING.spec.urn WORK_COMPLETED_URN = common_urns.monitoring_info_specs.WORK_COMPLETED.spec.urn @@ -72,11 +75,13 @@ LATEST_INT64_TYPE = common_urns.monitoring_info_types.LATEST_INT64_TYPE.urn PROGRESS_TYPE = common_urns.monitoring_info_types.PROGRESS_TYPE.urn STRING_SET_TYPE = common_urns.monitoring_info_types.SET_STRING_TYPE.urn +BOUNDED_TRIE_TYPE = common_urns.monitoring_info_types.BOUNDED_TRIE_TYPE.urn COUNTER_TYPES = set([SUM_INT64_TYPE]) DISTRIBUTION_TYPES = set([DISTRIBUTION_INT64_TYPE]) GAUGE_TYPES = set([LATEST_INT64_TYPE]) STRING_SET_TYPES = set([STRING_SET_TYPE]) +BOUNDED_TRIE_TYPES = set([BOUNDED_TRIE_TYPE]) # TODO(migryz) extract values from beam_fn_api.proto::MonitoringInfoLabels PCOLLECTION_LABEL = ( @@ -320,6 +325,23 @@ def user_set_string(namespace, name, metric, ptransform=None): USER_STRING_SET_URN, STRING_SET_TYPE, metric, labels) +def user_bounded_trie(namespace, name, metric, ptransform=None): + """Return the string set monitoring info for the URN, metric and labels. + + Args: + namespace: User-defined namespace of BoundedTrie. + name: Name of BoundedTrie. + metric: The BoundedTrieData representing the metrics. + ptransform: The ptransform id used as a label. + """ + labels = create_labels(ptransform=ptransform, namespace=namespace, name=name) + return create_monitoring_info( + USER_BOUNDED_TRIE_URN, + BOUNDED_TRIE_TYPE, + metric.to_proto().SerializeToString(), + labels) + + def create_monitoring_info( urn, type_urn, payload, labels=None) -> metrics_pb2.MonitoringInfo: """Return the gauge monitoring info for the URN, type, metric and labels. From cd02853f7fcf085136909353d5b6fc1c3cbe9879 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Thu, 5 Dec 2024 16:16:16 +0100 Subject: [PATCH 04/33] [infra] #33309 automatically apply spotless --- build.gradle.kts | 16 ++++++++++++++++ runners/flink/flink_runner.gradle | 5 +++++ 2 files changed, 21 insertions(+) diff --git a/build.gradle.kts b/build.gradle.kts index d96e77a4c78c..0adb29058479 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -647,6 +647,22 @@ tasks.register("checkSetup") { dependsOn(":examples:java:wordCount") } +// if not disabled make spotlessApply dependency of compileJava and compileTestJava +val disableSpotlessCheck: String by project +val isSpotlessDisabled = project.hasProperty("disableSpotlessCheck") && + disableSpotlessCheck == "true" +if (!isSpotlessDisabled) { + subprojects { + afterEvaluate { + tasks.findByName("spotlessApply")?.let { + listOf("compileJava", "compileTestJava").forEach { + t -> tasks.findByName(t)?.let { f -> f.dependsOn("spotlessApply") } + } + } + } + } +} + // Generates external transform config project.tasks.register("generateExternalTransformsConfig") { dependsOn(":sdks:python:generateExternalTransformsConfig") diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index d13e1c5faf6e..be39d4e0b012 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -422,3 +422,8 @@ createPipelineOptionsTableTask('Python') // Update the pipeline options documentation before running the tests test.dependsOn(generatePipelineOptionsTableJava) test.dependsOn(generatePipelineOptionsTablePython) + +// delegate spotlessApply to :runners:flink:spotlessApply +tasks.named("spotlessApply") { + dependsOn ":runners:flink:spotlessApply" +} From 0dae9c99427bfe484c26c138854d71ff9bfdeea3 Mon Sep 17 00:00:00 2001 From: Hai Joey Tran Date: Fri, 6 Dec 2024 20:56:38 -0500 Subject: [PATCH 05/33] Polish FlatMapTuple and MapTuple docstrings (#33316) --- sdks/python/apache_beam/transforms/core.py | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 4d1678d72a69..b420d1d66d09 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -2117,15 +2117,13 @@ def MapTuple(fn, *args, **kwargs): # pylint: disable=invalid-name r""":func:`MapTuple` is like :func:`Map` but expects tuple inputs and flattens them into multiple input arguments. - beam.MapTuple(lambda a, b, ...: ...) - In other words - beam.MapTuple(fn) + "SwapKV" >> beam.Map(lambda kv: (kv[1], kv[0])) is equivalent to - beam.Map(lambda element, ...: fn(\*element, ...)) + "SwapKV" >> beam.MapTuple(lambda k, v: (v, k)) This can be useful when processing a PCollection of tuples (e.g. key-value pairs). @@ -2191,19 +2189,13 @@ def FlatMapTuple(fn, *args, **kwargs): # pylint: disable=invalid-name r""":func:`FlatMapTuple` is like :func:`FlatMap` but expects tuple inputs and flattens them into multiple input arguments. - beam.FlatMapTuple(lambda a, b, ...: ...) - - is equivalent to Python 2 - - beam.FlatMap(lambda (a, b, ...), ...: ...) - In other words - beam.FlatMapTuple(fn) + beam.FlatMap(lambda start_end: range(start_end[0], start_end[1])) is equivalent to - beam.FlatMap(lambda element, ...: fn(\*element, ...)) + beam.FlatMapTuple(lambda start, end: range(start, end)) This can be useful when processing a PCollection of tuples (e.g. key-value pairs). From 85bff0d2c1adb743c4738f51428c88fb74532021 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 9 Dec 2024 11:59:38 -0500 Subject: [PATCH 06/33] Reapply "bump hadoop version (#33011)" (#33312) * Reapply "bump hadoop version (#33011)" (#33257) This reverts commit 7e25649b88b1ac08e48db28f8af09978b649da17. * Fix hbase and hcatalog test dependencies * Add missing pinned hadoop dependency version for compat test target --- .../beam_PostCommit_Java_Hadoop_Versions.json | 2 +- .../beam_PreCommit_Java_HBase_IO_Direct.json | 1 + CHANGES.md | 1 + .../org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- sdks/java/io/hadoop-common/build.gradle | 2 +- sdks/java/io/hadoop-file-system/build.gradle | 2 +- sdks/java/io/hadoop-format/build.gradle | 2 +- sdks/java/io/hbase/build.gradle | 9 ++------- sdks/java/io/hcatalog/build.gradle | 10 +++++++++- sdks/java/io/iceberg/build.gradle | 8 ++++++++ 10 files changed, 26 insertions(+), 13 deletions(-) create mode 100644 .github/trigger_files/beam_PreCommit_Java_HBase_IO_Direct.json diff --git a/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json b/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json index 920c8d132e4a..8784d0786c02 100644 --- a/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json +++ b/.github/trigger_files/beam_PostCommit_Java_Hadoop_Versions.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } \ No newline at end of file diff --git a/.github/trigger_files/beam_PreCommit_Java_HBase_IO_Direct.json b/.github/trigger_files/beam_PreCommit_Java_HBase_IO_Direct.json new file mode 100644 index 000000000000..0967ef424bce --- /dev/null +++ b/.github/trigger_files/beam_PreCommit_Java_HBase_IO_Direct.json @@ -0,0 +1 @@ +{} diff --git a/CHANGES.md b/CHANGES.md index 1b943a99f8a0..dbadd588ae3f 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -64,6 +64,7 @@ * gcs-connector config options can be set via GcsOptions (Java) ([#32769](https://github.com/apache/beam/pull/32769)). * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Upgraded the default version of Hadoop dependencies to 3.4.1. Hadoop 2.10.2 is still supported (Java) ([#33011](https://github.com/apache/beam/issues/33011)). ## New Features / Improvements diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 84c7c3ecfd4a..2abd43a5d4cc 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -614,7 +614,7 @@ class BeamModulePlugin implements Plugin { // [bomupgrader] determined by: io.grpc:grpc-netty, consistent with: google_cloud_platform_libraries_bom def grpc_version = "1.67.1" def guava_version = "33.1.0-jre" - def hadoop_version = "2.10.2" + def hadoop_version = "3.4.1" def hamcrest_version = "2.1" def influxdb_version = "2.19" def httpclient_version = "4.5.13" diff --git a/sdks/java/io/hadoop-common/build.gradle b/sdks/java/io/hadoop-common/build.gradle index b0303d29ff98..4375001ffa81 100644 --- a/sdks/java/io/hadoop-common/build.gradle +++ b/sdks/java/io/hadoop-common/build.gradle @@ -28,7 +28,7 @@ def hadoopVersions = [ "2102": "2.10.2", "324": "3.2.4", "336": "3.3.6", - "341": "3.4.1", + // "341": "3.4.1", // tests already exercised on the default version ] hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} diff --git a/sdks/java/io/hadoop-file-system/build.gradle b/sdks/java/io/hadoop-file-system/build.gradle index fafa8b5c7e34..b4ebbfa08c5e 100644 --- a/sdks/java/io/hadoop-file-system/build.gradle +++ b/sdks/java/io/hadoop-file-system/build.gradle @@ -29,7 +29,7 @@ def hadoopVersions = [ "2102": "2.10.2", "324": "3.2.4", "336": "3.3.6", - "341": "3.4.1", + // "341": "3.4.1", // tests already exercised on the default version ] hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle index 4664005a1fc8..73fc44a0f311 100644 --- a/sdks/java/io/hadoop-format/build.gradle +++ b/sdks/java/io/hadoop-format/build.gradle @@ -33,7 +33,7 @@ def hadoopVersions = [ "2102": "2.10.2", "324": "3.2.4", "336": "3.3.6", - "341": "3.4.1", + // "341": "3.4.1", // tests already exercised on the default version ] hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} diff --git a/sdks/java/io/hbase/build.gradle b/sdks/java/io/hbase/build.gradle index d85c0fc610bb..07014f2d5e3b 100644 --- a/sdks/java/io/hbase/build.gradle +++ b/sdks/java/io/hbase/build.gradle @@ -34,7 +34,7 @@ test { jvmArgs "-Dtest.build.data.basedirectory=build/test-data" } -def hbase_version = "2.5.5" +def hbase_version = "2.6.1-hadoop3" dependencies { implementation library.java.vendored_guava_32_1_2_jre @@ -46,12 +46,7 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit testImplementation library.java.hamcrest - testImplementation library.java.hadoop_minicluster - testImplementation library.java.hadoop_hdfs - testImplementation library.java.hadoop_common + // shaded-testing-utils has shaded all Hadoop/HBase dependencies testImplementation("org.apache.hbase:hbase-shaded-testing-util:$hbase_version") - testImplementation "org.apache.hbase:hbase-hadoop-compat:$hbase_version:tests" - testImplementation "org.apache.hbase:hbase-hadoop2-compat:$hbase_version:tests" testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } - diff --git a/sdks/java/io/hcatalog/build.gradle b/sdks/java/io/hcatalog/build.gradle index 364c10fa738b..d07904f3465e 100644 --- a/sdks/java/io/hcatalog/build.gradle +++ b/sdks/java/io/hcatalog/build.gradle @@ -33,7 +33,7 @@ def hadoopVersions = [ "2102": "2.10.2", "324": "3.2.4", "336": "3.3.6", - "341": "3.4.1", + // "341": "3.4.1", // tests already exercised on the default version ] hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} @@ -71,13 +71,21 @@ dependencies { testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") hadoopVersions.each {kv -> "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-common:$kv.value" + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-hdfs:$kv.value" + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-hdfs-client:$kv.value" + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-mapreduce-client-core:$kv.value" } } hadoopVersions.each {kv -> configurations."hadoopVersion$kv.key" { resolutionStrategy { + force "org.apache.hadoop:hadoop-client:$kv.value" force "org.apache.hadoop:hadoop-common:$kv.value" + force "org.apache.hadoop:hadoop-mapreduce-client-core:$kv.value" + force "org.apache.hadoop:hadoop-minicluster:$kv.value" + force "org.apache.hadoop:hadoop-hdfs:$kv.value" + force "org.apache.hadoop:hadoop-hdfs-client:$kv.value" } } } diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index a2d192b67208..fa1e2426ce69 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -71,6 +71,9 @@ dependencies { testRuntimeOnly project(path: ":runners:google-cloud-dataflow-java") hadoopVersions.each {kv -> "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-client:$kv.value" + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-minicluster:$kv.value" + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-hdfs-client:$kv.value" + "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-mapreduce-client-core:$kv.value" } } @@ -78,6 +81,11 @@ hadoopVersions.each {kv -> configurations."hadoopVersion$kv.key" { resolutionStrategy { force "org.apache.hadoop:hadoop-client:$kv.value" + force "org.apache.hadoop:hadoop-common:$kv.value" + force "org.apache.hadoop:hadoop-mapreduce-client-core:$kv.value" + force "org.apache.hadoop:hadoop-minicluster:$kv.value" + force "org.apache.hadoop:hadoop-hdfs:$kv.value" + force "org.apache.hadoop:hadoop-hdfs-client:$kv.value" } } } From a806c0ed3a9ead3b7738ae94bf323865abbd5e73 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Mon, 9 Dec 2024 13:20:12 -0500 Subject: [PATCH 07/33] Add Dataflow Cost Benchmark framework to Beam Python (#33297) * initial benchmark framework code * Implement Dataflow cost benchmark framework + add wordcount example * formatting * move to base wordcount instead * add comment for pipeline execution in wordcount --- ...rdcount_Python_Cost_Benchmark_Dataflow.yml | 91 ++++++++++++++ .../python_wordcount.txt | 28 +++++ sdks/python/apache_beam/examples/wordcount.py | 39 +++--- .../testing/benchmarks/wordcount/__init__.py | 16 +++ .../testing/benchmarks/wordcount/wordcount.py | 39 ++++++ .../load_tests/dataflow_cost_benchmark.py | 113 ++++++++++++++++++ .../load_tests/dataflow_cost_consts.py | 59 +++++++++ 7 files changed, 368 insertions(+), 17 deletions(-) create mode 100644 .github/workflows/beam_Wordcount_Python_Cost_Benchmark_Dataflow.yml create mode 100644 .github/workflows/cost-benchmarks-pipeline-options/python_wordcount.txt create mode 100644 sdks/python/apache_beam/testing/benchmarks/wordcount/__init__.py create mode 100644 sdks/python/apache_beam/testing/benchmarks/wordcount/wordcount.py create mode 100644 sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py create mode 100644 sdks/python/apache_beam/testing/load_tests/dataflow_cost_consts.py diff --git a/.github/workflows/beam_Wordcount_Python_Cost_Benchmark_Dataflow.yml b/.github/workflows/beam_Wordcount_Python_Cost_Benchmark_Dataflow.yml new file mode 100644 index 000000000000..51d1005affbc --- /dev/null +++ b/.github/workflows/beam_Wordcount_Python_Cost_Benchmark_Dataflow.yml @@ -0,0 +1,91 @@ +# 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. + +name: Wordcount Python Cost Benchmarks Dataflow + +on: + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} + INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} + +jobs: + beam_Inference_Python_Benchmarks_Dataflow: + if: | + github.event_name == 'workflow_dispatch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 900 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Wordcount_Python_Cost_Benchmarks_Dataflow"] + job_phrase: ["Run Wordcount Cost Benchmark"] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup Python environment + uses: ./.github/actions/setup-environment-action + with: + python-version: '3.10' + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/cost-benchmarks-pipeline-options/python_wordcount.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + - name: run wordcount on Dataflow Python + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 30 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.wordcount.wordcount \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.10 \ + '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_1 }} --job_name=benchmark-tests-wordcount-python-${{env.NOW_UTC}} --output=gs://temp-storage-for-end-to-end-tests/wordcount/result_wordcount-${{env.NOW_UTC}}.txt' \ \ No newline at end of file diff --git a/.github/workflows/cost-benchmarks-pipeline-options/python_wordcount.txt b/.github/workflows/cost-benchmarks-pipeline-options/python_wordcount.txt new file mode 100644 index 000000000000..424936ddad97 --- /dev/null +++ b/.github/workflows/cost-benchmarks-pipeline-options/python_wordcount.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--machine_type=n1-standard-2 +--num_workers=1 +--disk_size_gb=50 +--autoscaling_algorithm=NONE +--input_options={} +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=beam_run_inference +--metrics_table=python_wordcount +--runner=DataflowRunner \ No newline at end of file diff --git a/sdks/python/apache_beam/examples/wordcount.py b/sdks/python/apache_beam/examples/wordcount.py index 31407aec6c40..a9138647581c 100644 --- a/sdks/python/apache_beam/examples/wordcount.py +++ b/sdks/python/apache_beam/examples/wordcount.py @@ -45,6 +45,7 @@ from apache_beam.io import WriteToText from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.runners.runner import PipelineResult class WordExtractingDoFn(beam.DoFn): @@ -63,7 +64,7 @@ def process(self, element): return re.findall(r'[\w\']+', element, re.UNICODE) -def run(argv=None, save_main_session=True): +def run(argv=None, save_main_session=True) -> PipelineResult: """Main entry point; defines and runs the wordcount pipeline.""" parser = argparse.ArgumentParser() parser.add_argument( @@ -83,27 +84,31 @@ def run(argv=None, save_main_session=True): pipeline_options = PipelineOptions(pipeline_args) pipeline_options.view_as(SetupOptions).save_main_session = save_main_session - # The pipeline will be run on exiting the with block. - with beam.Pipeline(options=pipeline_options) as p: + pipeline = beam.Pipeline(options=pipeline_options) - # Read the text file[pattern] into a PCollection. - lines = p | 'Read' >> ReadFromText(known_args.input) + # Read the text file[pattern] into a PCollection. + lines = pipeline | 'Read' >> ReadFromText(known_args.input) - counts = ( - lines - | 'Split' >> (beam.ParDo(WordExtractingDoFn()).with_output_types(str)) - | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) - | 'GroupAndSum' >> beam.CombinePerKey(sum)) + counts = ( + lines + | 'Split' >> (beam.ParDo(WordExtractingDoFn()).with_output_types(str)) + | 'PairWithOne' >> beam.Map(lambda x: (x, 1)) + | 'GroupAndSum' >> beam.CombinePerKey(sum)) - # Format the counts into a PCollection of strings. - def format_result(word, count): - return '%s: %d' % (word, count) + # Format the counts into a PCollection of strings. + def format_result(word, count): + return '%s: %d' % (word, count) - output = counts | 'Format' >> beam.MapTuple(format_result) + output = counts | 'Format' >> beam.MapTuple(format_result) - # Write the output using a "Write" transform that has side effects. - # pylint: disable=expression-not-assigned - output | 'Write' >> WriteToText(known_args.output) + # Write the output using a "Write" transform that has side effects. + # pylint: disable=expression-not-assigned + output | 'Write' >> WriteToText(known_args.output) + + # Execute the pipeline and return the result. + result = pipeline.run() + result.wait_until_finish() + return result if __name__ == '__main__': diff --git a/sdks/python/apache_beam/testing/benchmarks/wordcount/__init__.py b/sdks/python/apache_beam/testing/benchmarks/wordcount/__init__.py new file mode 100644 index 000000000000..cce3acad34a4 --- /dev/null +++ b/sdks/python/apache_beam/testing/benchmarks/wordcount/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/sdks/python/apache_beam/testing/benchmarks/wordcount/wordcount.py b/sdks/python/apache_beam/testing/benchmarks/wordcount/wordcount.py new file mode 100644 index 000000000000..513ede47e80a --- /dev/null +++ b/sdks/python/apache_beam/testing/benchmarks/wordcount/wordcount.py @@ -0,0 +1,39 @@ +# +# 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. +# +# pytype: skip-file + +import logging + +from apache_beam.examples import wordcount +from apache_beam.testing.load_tests.dataflow_cost_benchmark import DataflowCostBenchmark + + +class WordcountCostBenchmark(DataflowCostBenchmark): + def __init__(self): + super().__init__() + + def test(self): + extra_opts = {} + extra_opts['output'] = self.pipeline.get_option('output_file') + self.result = wordcount.run( + self.pipeline.get_full_options_as_args(**extra_opts), + save_main_session=False) + + +if __name__ == '__main__': + logging.basicConfig(level=logging.INFO) + WordcountCostBenchmark().run() diff --git a/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py b/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py new file mode 100644 index 000000000000..b60af1249756 --- /dev/null +++ b/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py @@ -0,0 +1,113 @@ +# +# 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. + +# pytype: skip-file + +import logging +import time +from typing import Any +from typing import Optional + +import apache_beam.testing.load_tests.dataflow_cost_consts as costs +from apache_beam.metrics.execution import MetricResult +from apache_beam.runners.dataflow.dataflow_runner import DataflowPipelineResult +from apache_beam.runners.runner import PipelineState +from apache_beam.testing.load_tests.load_test import LoadTest + + +class DataflowCostBenchmark(LoadTest): + """Base class for Dataflow performance tests which export metrics to + external databases: BigQuery or/and InfluxDB. Calculates the expected cost + for running the job on Dataflow in region us-central1. + + Refer to :class:`~apache_beam.testing.load_tests.LoadTestOptions` for more + information on the required pipeline options. + + If using InfluxDB with Basic HTTP authentication enabled, provide the + following environment options: `INFLUXDB_USER` and `INFLUXDB_USER_PASSWORD`. + + If the hardware configuration for the job includes use of a GPU, please + specify the version in use with the Accelerator enumeration. This is used to + calculate the cost of the job later, as different accelerators have different + billing rates per hour of use. + """ + def __init__( + self, + metrics_namespace: Optional[str] = None, + is_streaming: bool = False, + gpu: Optional[costs.Accelerator] = None): + self.is_streaming = is_streaming + self.gpu = gpu + super().__init__(metrics_namespace=metrics_namespace) + + def run(self): + try: + self.test() + if not hasattr(self, 'result'): + self.result = self.pipeline.run() + # Defaults to waiting forever unless timeout has been set + state = self.result.wait_until_finish(duration=self.timeout_ms) + assert state != PipelineState.FAILED + logging.info( + 'Pipeline complete, sleeping for 4 minutes to allow resource ' + 'metrics to populate.') + time.sleep(240) + self.extra_metrics = self._retrieve_cost_metrics(self.result) + self._metrics_monitor.publish_metrics(self.result, self.extra_metrics) + finally: + self.cleanup() + + def _retrieve_cost_metrics(self, + result: DataflowPipelineResult) -> dict[str, Any]: + job_id = result.job_id() + metrics = result.metrics().all_metrics(job_id) + metrics_dict = self._process_metrics_list(metrics) + logging.info(metrics_dict) + cost = 0.0 + if (self.is_streaming): + cost += metrics_dict.get( + "TotalVcpuTime", 0.0) / 3600 * costs.VCPU_PER_HR_STREAMING + cost += ( + metrics_dict.get("TotalMemoryUsage", 0.0) / + 1000) / 3600 * costs.MEM_PER_GB_HR_STREAMING + cost += metrics_dict.get( + "TotalStreamingDataProcessed", 0.0) * costs.SHUFFLE_PER_GB_STREAMING + else: + cost += metrics_dict.get( + "TotalVcpuTime", 0.0) / 3600 * costs.VCPU_PER_HR_BATCH + cost += ( + metrics_dict.get("TotalMemoryUsage", 0.0) / + 1000) / 3600 * costs.MEM_PER_GB_HR_BATCH + cost += metrics_dict.get( + "TotalStreamingDataProcessed", 0.0) * costs.SHUFFLE_PER_GB_BATCH + if (self.gpu): + rate = costs.ACCELERATOR_TO_COST[self.gpu] + cost += metrics_dict.get("TotalGpuTime", 0.0) / 3600 * rate + cost += metrics_dict.get("TotalPdUsage", 0.0) / 3600 * costs.PD_PER_GB_HR + cost += metrics_dict.get( + "TotalSsdUsage", 0.0) / 3600 * costs.PD_SSD_PER_GB_HR + metrics_dict["EstimatedCost"] = cost + return metrics_dict + + def _process_metrics_list(self, + metrics: list[MetricResult]) -> dict[str, Any]: + system_metrics = {} + for entry in metrics: + metric_key = entry.key + metric = metric_key.metric + if metric_key.step == '' and metric.namespace == 'dataflow/v1b3': + system_metrics[metric.name] = entry.committed + return system_metrics diff --git a/sdks/python/apache_beam/testing/load_tests/dataflow_cost_consts.py b/sdks/python/apache_beam/testing/load_tests/dataflow_cost_consts.py new file mode 100644 index 000000000000..f291991b48bb --- /dev/null +++ b/sdks/python/apache_beam/testing/load_tests/dataflow_cost_consts.py @@ -0,0 +1,59 @@ +# +# 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. + +# These values are Dataflow costs for running jobs in us-central1. +# The cost values are found at https://cloud.google.com/dataflow/pricing + +from enum import Enum + +VCPU_PER_HR_BATCH = 0.056 +VCPU_PER_HR_STREAMING = 0.069 +MEM_PER_GB_HR_BATCH = 0.003557 +MEM_PER_GB_HR_STREAMING = 0.0035557 +PD_PER_GB_HR = 0.000054 +PD_SSD_PER_GB_HR = 0.000298 +SHUFFLE_PER_GB_BATCH = 0.011 +SHUFFLE_PER_GB_STREAMING = 0.018 + +# GPU Resource Pricing +P100_PER_GPU_PER_HOUR = 1.752 +V100_PER_GPU_PER_HOUR = 2.976 +T4_PER_GPU_PER_HOUR = 0.42 +P4_PER_GPU_PER_HOUR = 0.72 +L4_PER_GPU_PER_HOUR = 0.672 +A100_40GB_PER_GPU_PER_HOUR = 3.72 +A100_80GB_PER_GPU_PER_HOUR = 4.7137 + + +class Accelerator(Enum): + P100 = 1 + V100 = 2 + T4 = 3 + P4 = 4 + L4 = 5 + A100_40GB = 6 + A100_80GB = 7 + + +ACCELERATOR_TO_COST: dict[Accelerator, float] = { + Accelerator.P100: P100_PER_GPU_PER_HOUR, + Accelerator.V100: V100_PER_GPU_PER_HOUR, + Accelerator.T4: T4_PER_GPU_PER_HOUR, + Accelerator.P4: P4_PER_GPU_PER_HOUR, + Accelerator.L4: L4_PER_GPU_PER_HOUR, + Accelerator.A100_40GB: A100_40GB_PER_GPU_PER_HOUR, + Accelerator.A100_80GB: A100_80GB_PER_GPU_PER_HOUR, +} From 354e8e312e331d836309906dd5d8b6256499caf8 Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 10 Dec 2024 09:58:30 -0500 Subject: [PATCH 08/33] Replace None metric values with 0.0 in Dataflow Cost Benchmark (#33336) --- .../apache_beam/testing/load_tests/dataflow_cost_benchmark.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py b/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py index b60af1249756..96a1cd31e298 100644 --- a/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py +++ b/sdks/python/apache_beam/testing/load_tests/dataflow_cost_benchmark.py @@ -109,5 +109,7 @@ def _process_metrics_list(self, metric_key = entry.key metric = metric_key.metric if metric_key.step == '' and metric.namespace == 'dataflow/v1b3': + if entry.committed is None: + entry.committed = 0.0 system_metrics[metric.name] = entry.committed return system_metrics From 6d8e02e262951f26e8a4e835df462a219155bc11 Mon Sep 17 00:00:00 2001 From: Michel Davit Date: Tue, 10 Dec 2024 18:36:44 +0100 Subject: [PATCH 09/33] [java] BQ: add missing avro conversions to BQ TableRow (#33221) * [java] BQ: add missing avro conversions to BQ TableRow Avro float fields can be used to write BQ FLOAT columns. Add TableRow conversion for such field. Adding conversion for aveo 1.10+ logical types local-timestamp-millis and local-timestam-micros. * Rework tests * Add map and fixed types conversion * Fix checkstyle * Use valid parameters * Test record nullable field --- .../io/gcp/bigquery/BigQueryAvroUtils.java | 112 +- .../gcp/bigquery/BigQueryAvroUtilsTest.java | 1037 +++++++++++------ 2 files changed, 746 insertions(+), 403 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java index cddde05b194c..1af44ba7a012 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java @@ -34,6 +34,8 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; import org.apache.avro.Conversions; import org.apache.avro.LogicalType; @@ -50,14 +52,14 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -/** - * A set of utilities for working with Avro files. - * - *

These utilities are based on the Avro - * 1.8.1 specification. - */ +/** A set of utilities for working with Avro files. */ class BigQueryAvroUtils { + private static final String VERSION_AVRO = + Optional.ofNullable(Schema.class.getPackage()) + .map(Package::getImplementationVersion) + .orElse(""); + // org.apache.avro.LogicalType static class DateTimeLogicalType extends LogicalType { public DateTimeLogicalType() { @@ -74,6 +76,8 @@ public DateTimeLogicalType() { * export * @see BQ * avro storage + * @see BQ avro + * load */ static Schema getPrimitiveType(TableFieldSchema schema, Boolean useAvroLogicalTypes) { String bqType = schema.getType(); @@ -116,6 +120,9 @@ static Schema getPrimitiveType(TableFieldSchema schema, Boolean useAvroLogicalTy } case "DATETIME": if (useAvroLogicalTypes) { + // BQ export uses a custom logical type + // TODO for load/storage use + // LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType()) return DATETIME_LOGICAL_TYPE.addToSchema(SchemaBuilder.builder().stringType()); } else { return SchemaBuilder.builder().stringBuilder().prop("sqlType", bqType).endString(); @@ -158,6 +165,12 @@ static Schema getPrimitiveType(TableFieldSchema schema, Boolean useAvroLogicalTy @VisibleForTesting static String formatTimestamp(Long timestampMicro) { + String dateTime = formatDatetime(timestampMicro); + return dateTime + " UTC"; + } + + @VisibleForTesting + static String formatDatetime(Long timestampMicro) { // timestampMicro is in "microseconds since epoch" format, // e.g., 1452062291123456L means "2016-01-06 06:38:11.123456 UTC". // Separate into seconds and microseconds. @@ -168,11 +181,13 @@ static String formatTimestamp(Long timestampMicro) { timestampSec -= 1; } String dayAndTime = DATE_AND_SECONDS_FORMATTER.print(timestampSec * 1000); - if (micros == 0) { - return String.format("%s UTC", dayAndTime); + return dayAndTime; + } else if (micros % 1000 == 0) { + return String.format("%s.%03d", dayAndTime, micros / 1000); + } else { + return String.format("%s.%06d", dayAndTime, micros); } - return String.format("%s.%06d UTC", dayAndTime, micros); } /** @@ -274,8 +289,7 @@ static TableRow convertGenericRecordToTableRow(GenericRecord record) { case UNION: return convertNullableField(name, schema, v); case MAP: - throw new UnsupportedOperationException( - String.format("Unexpected Avro field schema type %s for field named %s", type, name)); + return convertMapField(name, schema, v); default: return convertRequiredField(name, schema, v); } @@ -296,6 +310,26 @@ private static List convertRepeatedField(String name, Schema elementType return values; } + private static List convertMapField(String name, Schema map, Object v) { + // Avro maps are represented as key/value RECORD. + if (v == null) { + // Handle the case of an empty map. + return new ArrayList<>(); + } + + Schema type = map.getValueType(); + Map elements = (Map) v; + ArrayList values = new ArrayList<>(); + for (Map.Entry element : elements.entrySet()) { + TableRow row = + new TableRow() + .set("key", element.getKey()) + .set("value", convertRequiredField(name, type, element.getValue())); + values.add(row); + } + return values; + } + private static Object convertRequiredField(String name, Schema schema, Object v) { // REQUIRED fields are represented as the corresponding Avro types. For example, a BigQuery // INTEGER type maps to an Avro LONG type. @@ -305,45 +339,83 @@ private static Object convertRequiredField(String name, Schema schema, Object v) LogicalType logicalType = schema.getLogicalType(); switch (type) { case BOOLEAN: - // SQL types BOOL, BOOLEAN + // SQL type BOOL (BOOLEAN) return v; case INT: if (logicalType instanceof LogicalTypes.Date) { - // SQL types DATE + // SQL type DATE + // ideally LocalDate but TableRowJsonCoder encodes as String return formatDate((Integer) v); + } else if (logicalType instanceof LogicalTypes.TimeMillis) { + // Write only: SQL type TIME + // ideally LocalTime but TableRowJsonCoder encodes as String + return formatTime(((Integer) v) * 1000L); } else { - throw new UnsupportedOperationException( - String.format("Unexpected Avro field schema type %s for field named %s", type, name)); + // Write only: SQL type INT64 (INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT) + // ideally Integer but keep consistency with BQ JSON export that uses String + return ((Integer) v).toString(); } case LONG: if (logicalType instanceof LogicalTypes.TimeMicros) { - // SQL types TIME + // SQL type TIME + // ideally LocalTime but TableRowJsonCoder encodes as String return formatTime((Long) v); + } else if (logicalType instanceof LogicalTypes.TimestampMillis) { + // Write only: SQL type TIMESTAMP + // ideally Instant but TableRowJsonCoder encodes as String + return formatTimestamp((Long) v * 1000L); } else if (logicalType instanceof LogicalTypes.TimestampMicros) { - // SQL types TIMESTAMP + // SQL type TIMESTAMP + // ideally Instant but TableRowJsonCoder encodes as String return formatTimestamp((Long) v); + } else if (!(VERSION_AVRO.startsWith("1.8") || VERSION_AVRO.startsWith("1.9")) + && logicalType instanceof LogicalTypes.LocalTimestampMillis) { + // Write only: SQL type DATETIME + // ideally LocalDateTime but TableRowJsonCoder encodes as String + return formatDatetime(((Long) v) * 1000); + } else if (!(VERSION_AVRO.startsWith("1.8") || VERSION_AVRO.startsWith("1.9")) + && logicalType instanceof LogicalTypes.LocalTimestampMicros) { + // Write only: SQL type DATETIME + // ideally LocalDateTime but TableRowJsonCoder encodes as String + return formatDatetime((Long) v); } else { - // SQL types INT64 (INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT) + // SQL type INT64 (INT, SMALLINT, INTEGER, BIGINT, TINYINT, BYTEINT) + // ideally Long if in [2^53+1, 2^53-1] but keep consistency with BQ JSON export that uses + // String return ((Long) v).toString(); } + case FLOAT: + // Write only: SQL type FLOAT64 + // ideally Float but TableRowJsonCoder decodes as Double + return Double.valueOf(v.toString()); case DOUBLE: - // SQL types FLOAT64 + // SQL type FLOAT64 return v; case BYTES: if (logicalType instanceof LogicalTypes.Decimal) { // SQL tpe NUMERIC, BIGNUMERIC + // ideally BigDecimal but TableRowJsonCoder encodes as String return new Conversions.DecimalConversion() .fromBytes((ByteBuffer) v, schema, logicalType) .toString(); } else { - // SQL types BYTES + // SQL type BYTES + // ideally byte[] but TableRowJsonCoder encodes as String return BaseEncoding.base64().encode(((ByteBuffer) v).array()); } case STRING: // SQL types STRING, DATETIME, GEOGRAPHY, JSON // when not using logical type DATE, TIME too return v.toString(); + case ENUM: + // SQL types STRING + return v.toString(); + case FIXED: + // SQL type BYTES + // ideally byte[] but TableRowJsonCoder encodes as String + return BaseEncoding.base64().encode(((ByteBuffer) v).array()); case RECORD: + // SQL types RECORD return convertGenericRecordToTableRow((GenericRecord) v); default: throw new UnsupportedOperationException( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java index 662f2658eb6b..2333278a11f5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java @@ -28,23 +28,23 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.time.Instant; -import java.util.ArrayList; -import java.util.List; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; import org.apache.avro.Conversions; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; -import org.apache.avro.Schema.Type; +import org.apache.avro.SchemaBuilder; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.apache.avro.reflect.AvroSchema; -import org.apache.avro.reflect.Nullable; -import org.apache.avro.reflect.ReflectData; +import org.apache.avro.generic.GenericRecordBuilder; import org.apache.avro.util.Utf8; -import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; import org.junit.Test; @@ -54,363 +54,678 @@ /** Tests for {@link BigQueryAvroUtils}. */ @RunWith(JUnit4.class) public class BigQueryAvroUtilsTest { - private List subFields = - Lists.newArrayList( - new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE")); - /* - * Note that the quality and quantity fields do not have their mode set, so they should default - * to NULLABLE. This is an important test of BigQuery semantics. - * - * All the other fields we set in this function are required on the Schema response. - * - * See https://cloud.google.com/bigquery/docs/reference/v2/tables#schema - */ - private List fields = - Lists.newArrayList( - new TableFieldSchema().setName("number").setType("INTEGER").setMode("REQUIRED"), - new TableFieldSchema().setName("species").setType("STRING").setMode("NULLABLE"), - new TableFieldSchema().setName("quality").setType("FLOAT") /* default to NULLABLE */, - new TableFieldSchema().setName("quantity").setType("INTEGER") /* default to NULLABLE */, - new TableFieldSchema().setName("birthday").setType("TIMESTAMP").setMode("NULLABLE"), - new TableFieldSchema().setName("birthdayMoney").setType("NUMERIC").setMode("NULLABLE"), - new TableFieldSchema() - .setName("lotteryWinnings") - .setType("BIGNUMERIC") - .setMode("NULLABLE"), - new TableFieldSchema().setName("flighted").setType("BOOLEAN").setMode("NULLABLE"), - new TableFieldSchema().setName("sound").setType("BYTES").setMode("NULLABLE"), - new TableFieldSchema().setName("anniversaryDate").setType("DATE").setMode("NULLABLE"), - new TableFieldSchema() - .setName("anniversaryDatetime") - .setType("DATETIME") - .setMode("NULLABLE"), - new TableFieldSchema().setName("anniversaryTime").setType("TIME").setMode("NULLABLE"), - new TableFieldSchema() - .setName("scion") - .setType("RECORD") - .setMode("NULLABLE") - .setFields(subFields), - new TableFieldSchema() - .setName("associates") - .setType("RECORD") - .setMode("REPEATED") - .setFields(subFields), - new TableFieldSchema().setName("geoPositions").setType("GEOGRAPHY").setMode("NULLABLE")); - - private ByteBuffer convertToBytes(BigDecimal bigDecimal, int precision, int scale) { - LogicalType bigDecimalLogicalType = LogicalTypes.decimal(precision, scale); - return new Conversions.DecimalConversion().toBytes(bigDecimal, null, bigDecimalLogicalType); + + private TableSchema tableSchema(Function fn) { + TableFieldSchema column = new TableFieldSchema().setName("value"); + TableSchema tableSchema = new TableSchema(); + tableSchema.setFields(Lists.newArrayList(fn.apply(column))); + return tableSchema; + } + + private Schema avroSchema( + Function, SchemaBuilder.FieldAssembler> fn) { + return fn.apply( + SchemaBuilder.record("root") + .namespace("org.apache.beam.sdk.io.gcp.bigquery") + .doc("Translated Avro Schema for root") + .fields() + .name("value")) + .endRecord(); } + @SuppressWarnings("JavaInstantGetSecondsGetNano") @Test - public void testConvertGenericRecordToTableRow() throws Exception { - BigDecimal numeric = new BigDecimal("123456789.123456789"); - ByteBuffer numericBytes = convertToBytes(numeric, 38, 9); - BigDecimal bigNumeric = - new BigDecimal( - "578960446186580977117854925043439539266.34992332820282019728792003956564819967"); - ByteBuffer bigNumericBytes = convertToBytes(bigNumeric, 77, 38); - Schema avroSchema = ReflectData.get().getSchema(Bird.class); - - { - // Test nullable fields. - GenericRecord record = new GenericData.Record(avroSchema); - record.put("number", 5L); - TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record); - TableRow row = new TableRow().set("number", "5").set("associates", new ArrayList()); - assertEquals(row, convertedRow); - TableRow clonedRow = convertedRow.clone(); - assertEquals(convertedRow, clonedRow); - } - { - // Test type conversion for: - // INTEGER, FLOAT, NUMERIC, TIMESTAMP, BOOLEAN, BYTES, DATE, DATETIME, TIME. - GenericRecord record = new GenericData.Record(avroSchema); - byte[] soundBytes = "chirp,chirp".getBytes(StandardCharsets.UTF_8); - ByteBuffer soundByteBuffer = ByteBuffer.wrap(soundBytes); - soundByteBuffer.rewind(); - record.put("number", 5L); - record.put("quality", 5.0); - record.put("birthday", 5L); - record.put("birthdayMoney", numericBytes); - record.put("lotteryWinnings", bigNumericBytes); - record.put("flighted", Boolean.TRUE); - record.put("sound", soundByteBuffer); - record.put("anniversaryDate", new Utf8("2000-01-01")); - record.put("anniversaryDatetime", new String("2000-01-01 00:00:00.000005")); - record.put("anniversaryTime", new Utf8("00:00:00.000005")); - record.put("geoPositions", new String("LINESTRING(1 2, 3 4, 5 6, 7 8)")); - TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record); - TableRow row = - new TableRow() - .set("number", "5") - .set("birthday", "1970-01-01 00:00:00.000005 UTC") - .set("birthdayMoney", numeric.toString()) - .set("lotteryWinnings", bigNumeric.toString()) - .set("quality", 5.0) - .set("associates", new ArrayList()) - .set("flighted", Boolean.TRUE) - .set("sound", BaseEncoding.base64().encode(soundBytes)) - .set("anniversaryDate", "2000-01-01") - .set("anniversaryDatetime", "2000-01-01 00:00:00.000005") - .set("anniversaryTime", "00:00:00.000005") - .set("geoPositions", "LINESTRING(1 2, 3 4, 5 6, 7 8)"); - TableRow clonedRow = convertedRow.clone(); - assertEquals(convertedRow, clonedRow); - assertEquals(row, convertedRow); - } - { - // Test repeated fields. - Schema subBirdSchema = AvroCoder.of(Bird.SubBird.class).getSchema(); - GenericRecord nestedRecord = new GenericData.Record(subBirdSchema); - nestedRecord.put("species", "other"); - GenericRecord record = new GenericData.Record(avroSchema); - record.put("number", 5L); - record.put("associates", Lists.newArrayList(nestedRecord)); - record.put("birthdayMoney", numericBytes); - record.put("lotteryWinnings", bigNumericBytes); - TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record); - TableRow row = + public void testConvertGenericRecordToTableRow() { + { + // bool + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().booleanType().noDefault())) + .set("value", false) + .build(); + TableRow expected = new TableRow().set("value", false); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // int + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().intType().noDefault())) + .set("value", 5) + .build(); + TableRow expected = new TableRow().set("value", "5"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // long + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().longType().noDefault())) + .set("value", 5L) + .build(); + TableRow expected = new TableRow().set("value", "5"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // float + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().floatType().noDefault())) + .set("value", 5.5f) + .build(); + TableRow expected = new TableRow().set("value", 5.5); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // double + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().doubleType().noDefault())) + .set("value", 5.55) + .build(); + TableRow expected = new TableRow().set("value", 5.55); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // bytes + byte[] bytes = "chirp,chirp".getBytes(StandardCharsets.UTF_8); + ByteBuffer bb = ByteBuffer.wrap(bytes); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().bytesType().noDefault())) + .set("value", bb) + .build(); + TableRow expected = new TableRow().set("value", BaseEncoding.base64().encode(bytes)); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // string + Schema schema = avroSchema(f -> f.type().stringType().noDefault()); + GenericRecord record = new GenericRecordBuilder(schema).set("value", "test").build(); + GenericRecord utf8Record = + new GenericRecordBuilder(schema).set("value", new Utf8("test")).build(); + TableRow expected = new TableRow().set("value", "test"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + TableRow utf8Row = BigQueryAvroUtils.convertGenericRecordToTableRow(utf8Record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + assertEquals(expected, utf8Row); + assertEquals(expected, utf8Row.clone()); + } + + { + // decimal + LogicalType lt = LogicalTypes.decimal(38, 9); + Schema decimalType = lt.addToSchema(SchemaBuilder.builder().bytesType()); + BigDecimal bd = new BigDecimal("123456789.123456789"); + ByteBuffer bytes = new Conversions.DecimalConversion().toBytes(bd, null, lt); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(decimalType).noDefault())) + .set("value", bytes) + .build(); + TableRow expected = new TableRow().set("value", bd.toString()); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // date + LogicalType lt = LogicalTypes.date(); + Schema dateType = lt.addToSchema(SchemaBuilder.builder().intType()); + LocalDate date = LocalDate.of(2000, 1, 1); + int days = (int) date.toEpochDay(); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(dateType).noDefault())) + .set("value", days) + .build(); + TableRow expected = new TableRow().set("value", "2000-01-01"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // time-millis + LogicalType lt = LogicalTypes.timeMillis(); + Schema timeType = lt.addToSchema(SchemaBuilder.builder().intType()); + LocalTime time = LocalTime.of(1, 2, 3, 123456789); + int millis = (int) (time.toNanoOfDay() / 1000000); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(timeType).noDefault())) + .set("value", millis) + .build(); + TableRow expected = new TableRow().set("value", "01:02:03.123"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // time-micros + LogicalType lt = LogicalTypes.timeMicros(); + Schema timeType = lt.addToSchema(SchemaBuilder.builder().longType()); + LocalTime time = LocalTime.of(1, 2, 3, 123456789); + long micros = time.toNanoOfDay() / 1000; + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(timeType).noDefault())) + .set("value", micros) + .build(); + TableRow expected = new TableRow().set("value", "01:02:03.123456"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // local-timestamp-millis + LogicalType lt = LogicalTypes.localTimestampMillis(); + Schema timestampType = lt.addToSchema(SchemaBuilder.builder().longType()); + LocalDate date = LocalDate.of(2000, 1, 1); + LocalTime time = LocalTime.of(1, 2, 3, 123456789); + LocalDateTime ts = LocalDateTime.of(date, time); + long millis = ts.toInstant(ZoneOffset.UTC).toEpochMilli(); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(timestampType).noDefault())) + .set("value", millis) + .build(); + TableRow expected = new TableRow().set("value", "2000-01-01 01:02:03.123"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // local-timestamp-micros + LogicalType lt = LogicalTypes.localTimestampMicros(); + Schema timestampType = lt.addToSchema(SchemaBuilder.builder().longType()); + LocalDate date = LocalDate.of(2000, 1, 1); + LocalTime time = LocalTime.of(1, 2, 3, 123456789); + LocalDateTime ts = LocalDateTime.of(date, time); + long seconds = ts.toInstant(ZoneOffset.UTC).getEpochSecond(); + int nanos = ts.toInstant(ZoneOffset.UTC).getNano(); + long micros = seconds * 1000000 + (nanos / 1000); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(timestampType).noDefault())) + .set("value", micros) + .build(); + TableRow expected = new TableRow().set("value", "2000-01-01 01:02:03.123456"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // timestamp-micros + LogicalType lt = LogicalTypes.timestampMillis(); + Schema timestampType = lt.addToSchema(SchemaBuilder.builder().longType()); + LocalDate date = LocalDate.of(2000, 1, 1); + LocalTime time = LocalTime.of(1, 2, 3, 123456789); + LocalDateTime ts = LocalDateTime.of(date, time); + long millis = ts.toInstant(ZoneOffset.UTC).toEpochMilli(); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(timestampType).noDefault())) + .set("value", millis) + .build(); + TableRow expected = new TableRow().set("value", "2000-01-01 01:02:03.123 UTC"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // timestamp-millis + LogicalType lt = LogicalTypes.timestampMicros(); + Schema timestampType = lt.addToSchema(SchemaBuilder.builder().longType()); + LocalDate date = LocalDate.of(2000, 1, 1); + LocalTime time = LocalTime.of(1, 2, 3, 123456789); + LocalDateTime ts = LocalDateTime.of(date, time); + long seconds = ts.toInstant(ZoneOffset.UTC).getEpochSecond(); + int nanos = ts.toInstant(ZoneOffset.UTC).getNano(); + long micros = seconds * 1000000 + (nanos / 1000); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(timestampType).noDefault())) + .set("value", micros) + .build(); + TableRow expected = new TableRow().set("value", "2000-01-01 01:02:03.123456 UTC"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // enum + Schema enumSchema = SchemaBuilder.enumeration("color").symbols("red", "green", "blue"); + GenericData.EnumSymbol symbol = new GenericData.EnumSymbol(enumSchema, "RED"); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(enumSchema).noDefault())) + .set("value", symbol) + .build(); + TableRow expected = new TableRow().set("value", "RED"); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // fixed + UUID uuid = UUID.randomUUID(); + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + bb.rewind(); + byte[] bytes = bb.array(); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().fixed("uuid").size(16).noDefault())) + .set("value", bb) + .build(); + TableRow expected = new TableRow().set("value", BaseEncoding.base64().encode(bytes)); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // null + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().optional().booleanType())).build(); + TableRow expected = new TableRow(); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // array + GenericRecord record = + new GenericRecordBuilder( + avroSchema(f -> f.type().array().items().booleanType().noDefault())) + .set("value", Lists.newArrayList(true, false)) + .build(); + TableRow expected = new TableRow().set("value", Lists.newArrayList(true, false)); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // map + Map map = new HashMap<>(); + map.put("left", 1); + map.put("right", -1); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type().map().values().intType().noDefault())) + .set("value", map) + .build(); + TableRow expected = new TableRow() - .set("associates", Lists.newArrayList(new TableRow().set("species", "other"))) - .set("number", "5") - .set("birthdayMoney", numeric.toString()) - .set("lotteryWinnings", bigNumeric.toString()); - assertEquals(row, convertedRow); - TableRow clonedRow = convertedRow.clone(); - assertEquals(convertedRow, clonedRow); + .set( + "value", + Lists.newArrayList( + new TableRow().set("key", "left").set("value", "1"), + new TableRow().set("key", "right").set("value", "-1"))); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); + } + + { + // record + Schema subSchema = + SchemaBuilder.builder() + .record("record") + .fields() + .name("int") + .type() + .intType() + .noDefault() + .name("float") + .type() + .floatType() + .noDefault() + .endRecord(); + GenericRecord subRecord = + new GenericRecordBuilder(subSchema).set("int", 5).set("float", 5.5f).build(); + GenericRecord record = + new GenericRecordBuilder(avroSchema(f -> f.type(subSchema).noDefault())) + .set("value", subRecord) + .build(); + TableRow expected = + new TableRow().set("value", new TableRow().set("int", "5").set("float", 5.5)); + TableRow row = BigQueryAvroUtils.convertGenericRecordToTableRow(record); + + assertEquals(expected, row); + assertEquals(expected, row.clone()); } } @Test public void testConvertBigQuerySchemaToAvroSchema() { - TableSchema tableSchema = new TableSchema(); - tableSchema.setFields(fields); - Schema avroSchema = BigQueryAvroUtils.toGenericAvroSchema(tableSchema); + { + // REQUIRED + TableSchema tableSchema = tableSchema(f -> f.setType("BOOLEAN").setMode("REQUIRED")); + Schema expected = avroSchema(f -> f.type().booleanType().noDefault()); - assertThat(avroSchema.getField("number").schema(), equalTo(Schema.create(Type.LONG))); - assertThat( - avroSchema.getField("species").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.STRING)))); - assertThat( - avroSchema.getField("quality").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.DOUBLE)))); - assertThat( - avroSchema.getField("quantity").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.LONG)))); - assertThat( - avroSchema.getField("birthday").schema(), - equalTo( - Schema.createUnion( - Schema.create(Type.NULL), - LogicalTypes.timestampMicros().addToSchema(Schema.create(Type.LONG))))); - assertThat( - avroSchema.getField("birthdayMoney").schema(), - equalTo( - Schema.createUnion( - Schema.create(Type.NULL), - LogicalTypes.decimal(38, 9).addToSchema(Schema.create(Type.BYTES))))); - assertThat( - avroSchema.getField("lotteryWinnings").schema(), - equalTo( - Schema.createUnion( - Schema.create(Type.NULL), - LogicalTypes.decimal(77, 38).addToSchema(Schema.create(Type.BYTES))))); - assertThat( - avroSchema.getField("flighted").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.BOOLEAN)))); - assertThat( - avroSchema.getField("sound").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.BYTES)))); - Schema dateSchema = Schema.create(Type.INT); - LogicalTypes.date().addToSchema(dateSchema); - assertThat( - avroSchema.getField("anniversaryDate").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), dateSchema))); - Schema dateTimeSchema = Schema.create(Type.STRING); - BigQueryAvroUtils.DATETIME_LOGICAL_TYPE.addToSchema(dateTimeSchema); - assertThat( - avroSchema.getField("anniversaryDatetime").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), dateTimeSchema))); - Schema timeSchema = Schema.create(Type.LONG); - LogicalTypes.timeMicros().addToSchema(timeSchema); - assertThat( - avroSchema.getField("anniversaryTime").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), timeSchema))); - Schema geoSchema = Schema.create(Type.STRING); - geoSchema.addProp("sqlType", "GEOGRAPHY"); - assertThat( - avroSchema.getField("geoPositions").schema(), - equalTo(Schema.createUnion(Schema.create(Type.NULL), geoSchema))); - assertThat( - avroSchema.getField("scion").schema(), - equalTo( - Schema.createUnion( - Schema.create(Type.NULL), - Schema.createRecord( - "scion", - "Translated Avro Schema for scion", - "org.apache.beam.sdk.io.gcp.bigquery", - false, - ImmutableList.of( - new Field( - "species", - Schema.createUnion( - Schema.create(Type.NULL), Schema.create(Type.STRING)), - null, - (Object) null)))))); - assertThat( - avroSchema.getField("associates").schema(), - equalTo( - Schema.createArray( - Schema.createRecord( - "associates", - "Translated Avro Schema for associates", - "org.apache.beam.sdk.io.gcp.bigquery", - false, - ImmutableList.of( - new Field( - "species", - Schema.createUnion( - Schema.create(Type.NULL), Schema.create(Type.STRING)), - null, - (Object) null)))))); - } + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + } - @Test - public void testConvertBigQuerySchemaToAvroSchemaWithoutLogicalTypes() { - TableSchema tableSchema = new TableSchema(); - tableSchema.setFields(fields); - Schema avroSchema = BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false); + { + // NULLABLE + TableSchema tableSchema = tableSchema(f -> f.setType("BOOLEAN").setMode("NULLABLE")); + Schema expected = + avroSchema(f -> f.type().unionOf().nullType().and().booleanType().endUnion().noDefault()); - assertThat(avroSchema.getField("number").schema(), equalTo(Schema.create(Schema.Type.LONG))); - assertThat( - avroSchema.getField("species").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING)))); - assertThat( - avroSchema.getField("quality").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.DOUBLE)))); - assertThat( - avroSchema.getField("quantity").schema(), - equalTo( - Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG)))); - assertThat( - avroSchema.getField("birthday").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), - LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG))))); - assertThat( - avroSchema.getField("birthdayMoney").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), - LogicalTypes.decimal(38, 9).addToSchema(Schema.create(Schema.Type.BYTES))))); - assertThat( - avroSchema.getField("lotteryWinnings").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), - LogicalTypes.decimal(77, 38).addToSchema(Schema.create(Schema.Type.BYTES))))); - assertThat( - avroSchema.getField("flighted").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BOOLEAN)))); - assertThat( - avroSchema.getField("sound").schema(), - equalTo( - Schema.createUnion(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BYTES)))); - Schema dateSchema = Schema.create(Schema.Type.STRING); - dateSchema.addProp("sqlType", "DATE"); - assertThat( - avroSchema.getField("anniversaryDate").schema(), - equalTo(Schema.createUnion(Schema.create(Schema.Type.NULL), dateSchema))); - Schema dateTimeSchema = Schema.create(Schema.Type.STRING); - dateTimeSchema.addProp("sqlType", "DATETIME"); - assertThat( - avroSchema.getField("anniversaryDatetime").schema(), - equalTo(Schema.createUnion(Schema.create(Schema.Type.NULL), dateTimeSchema))); - Schema timeSchema = Schema.create(Schema.Type.STRING); - timeSchema.addProp("sqlType", "TIME"); - assertThat( - avroSchema.getField("anniversaryTime").schema(), - equalTo(Schema.createUnion(Schema.create(Schema.Type.NULL), timeSchema))); - Schema geoSchema = Schema.create(Type.STRING); - geoSchema.addProp("sqlType", "GEOGRAPHY"); - assertThat( - avroSchema.getField("geoPositions").schema(), - equalTo(Schema.createUnion(Schema.create(Schema.Type.NULL), geoSchema))); - assertThat( - avroSchema.getField("scion").schema(), - equalTo( - Schema.createUnion( - Schema.create(Schema.Type.NULL), - Schema.createRecord( - "scion", - "Translated Avro Schema for scion", - "org.apache.beam.sdk.io.gcp.bigquery", - false, - ImmutableList.of( - new Schema.Field( - "species", - Schema.createUnion( - Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING)), - null, - (Object) null)))))); - assertThat( - avroSchema.getField("associates").schema(), - equalTo( - Schema.createArray( - Schema.createRecord( - "associates", - "Translated Avro Schema for associates", - "org.apache.beam.sdk.io.gcp.bigquery", - false, - ImmutableList.of( - new Schema.Field( - "species", - Schema.createUnion( - Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING)), - null, - (Object) null)))))); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + } + + { + // default mode -> NULLABLE + TableSchema tableSchema = tableSchema(f -> f.setType("BOOLEAN")); + Schema expected = + avroSchema(f -> f.type().unionOf().nullType().and().booleanType().endUnion().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + } + + { + // REPEATED + TableSchema tableSchema = tableSchema(f -> f.setType("BOOLEAN").setMode("REPEATED")); + Schema expected = avroSchema(f -> f.type().array().items().booleanType().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + } + + { + // INTEGER + TableSchema tableSchema = tableSchema(f -> f.setType("INTEGER").setMode("REQUIRED")); + Schema expected = avroSchema(f -> f.type().longType().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + } + + { + // FLOAT + TableSchema tableSchema = tableSchema(f -> f.setType("FLOAT").setMode("REQUIRED")); + Schema expected = avroSchema(f -> f.type().doubleType().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // BYTES + TableSchema tableSchema = tableSchema(f -> f.setType("BYTES").setMode("REQUIRED")); + Schema expected = avroSchema(f -> f.type().bytesType().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // STRING + TableSchema tableSchema = tableSchema(f -> f.setType("STRING").setMode("REQUIRED")); + Schema expected = avroSchema(f -> f.type().stringType().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // NUMERIC + TableSchema tableSchema = tableSchema(f -> f.setType("NUMERIC").setMode("REQUIRED")); + Schema decimalType = + LogicalTypes.decimal(38, 9).addToSchema(SchemaBuilder.builder().bytesType()); + Schema expected = avroSchema(f -> f.type(decimalType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // NUMERIC with precision + TableSchema tableSchema = + tableSchema(f -> f.setType("NUMERIC").setPrecision(29L).setMode("REQUIRED")); + Schema decimalType = + LogicalTypes.decimal(29, 0).addToSchema(SchemaBuilder.builder().bytesType()); + Schema expected = avroSchema(f -> f.type(decimalType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // NUMERIC with precision and scale + TableSchema tableSchema = + tableSchema(f -> f.setType("NUMERIC").setPrecision(10L).setScale(9L).setMode("REQUIRED")); + Schema decimalType = + LogicalTypes.decimal(10, 9).addToSchema(SchemaBuilder.builder().bytesType()); + Schema expected = avroSchema(f -> f.type(decimalType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // BIGNUMERIC + TableSchema tableSchema = tableSchema(f -> f.setType("BIGNUMERIC").setMode("REQUIRED")); + Schema decimalType = + LogicalTypes.decimal(77, 38).addToSchema(SchemaBuilder.builder().bytesType()); + Schema expected = avroSchema(f -> f.type(decimalType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // BIGNUMERIC with precision + TableSchema tableSchema = + tableSchema(f -> f.setType("BIGNUMERIC").setPrecision(38L).setMode("REQUIRED")); + Schema decimalType = + LogicalTypes.decimal(38, 0).addToSchema(SchemaBuilder.builder().bytesType()); + Schema expected = avroSchema(f -> f.type(decimalType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // BIGNUMERIC with precision and scale + TableSchema tableSchema = + tableSchema( + f -> f.setType("BIGNUMERIC").setPrecision(39L).setScale(38L).setMode("REQUIRED")); + Schema decimalType = + LogicalTypes.decimal(39, 38).addToSchema(SchemaBuilder.builder().bytesType()); + Schema expected = avroSchema(f -> f.type(decimalType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // DATE + TableSchema tableSchema = tableSchema(f -> f.setType("DATE").setMode("REQUIRED")); + Schema dateType = LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType()); + Schema expected = avroSchema(f -> f.type(dateType).noDefault()); + Schema expectedExport = + avroSchema(f -> f.type().stringBuilder().prop("sqlType", "DATE").endString().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expectedExport, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // TIME + TableSchema tableSchema = tableSchema(f -> f.setType("TIME").setMode("REQUIRED")); + Schema timeType = LogicalTypes.timeMicros().addToSchema(SchemaBuilder.builder().longType()); + Schema expected = avroSchema(f -> f.type(timeType).noDefault()); + Schema expectedExport = + avroSchema(f -> f.type().stringBuilder().prop("sqlType", "TIME").endString().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expectedExport, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // DATETIME + TableSchema tableSchema = tableSchema(f -> f.setType("DATETIME").setMode("REQUIRED")); + Schema timeType = + BigQueryAvroUtils.DATETIME_LOGICAL_TYPE.addToSchema(SchemaBuilder.builder().stringType()); + Schema expected = avroSchema(f -> f.type(timeType).noDefault()); + Schema expectedExport = + avroSchema( + f -> f.type().stringBuilder().prop("sqlType", "DATETIME").endString().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expectedExport, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // TIMESTAMP + TableSchema tableSchema = tableSchema(f -> f.setType("TIMESTAMP").setMode("REQUIRED")); + Schema timestampType = + LogicalTypes.timestampMicros().addToSchema(SchemaBuilder.builder().longType()); + Schema expected = avroSchema(f -> f.type(timestampType).noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // GEOGRAPHY + TableSchema tableSchema = tableSchema(f -> f.setType("GEOGRAPHY").setMode("REQUIRED")); + Schema expected = + avroSchema( + f -> f.type().stringBuilder().prop("sqlType", "GEOGRAPHY").endString().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // JSON + TableSchema tableSchema = tableSchema(f -> f.setType("JSON").setMode("REQUIRED")); + Schema expected = + avroSchema(f -> f.type().stringBuilder().prop("sqlType", "JSON").endString().noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(tableSchema, false)); + } + + { + // STRUCT/RECORD + TableFieldSchema subInteger = + new TableFieldSchema().setName("int").setType("INTEGER").setMode("NULLABLE"); + TableFieldSchema subFloat = + new TableFieldSchema().setName("float").setType("FLOAT").setMode("REQUIRED"); + TableSchema structTableSchema = + tableSchema( + f -> + f.setType("STRUCT") + .setMode("REQUIRED") + .setFields(Lists.newArrayList(subInteger, subFloat))); + TableSchema recordTableSchema = + tableSchema( + f -> + f.setType("RECORD") + .setMode("REQUIRED") + .setFields(Lists.newArrayList(subInteger, subFloat))); + + Schema expected = + avroSchema( + f -> + f.type() + .record("value") + .fields() + .name("int") + .type() + .unionOf() + .nullType() + .and() + .longType() + .endUnion() + .noDefault() + .name("float") + .type() + .doubleType() + .noDefault() + .endRecord() + .noDefault()); + + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(structTableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(structTableSchema, false)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(recordTableSchema)); + assertEquals(expected, BigQueryAvroUtils.toGenericAvroSchema(recordTableSchema, false)); + } } @Test public void testFormatTimestamp() { - assertThat( - BigQueryAvroUtils.formatTimestamp(1452062291123456L), - equalTo("2016-01-06 06:38:11.123456 UTC")); + long micros = 1452062291123456L; + String expected = "2016-01-06 06:38:11.123456"; + assertThat(BigQueryAvroUtils.formatDatetime(micros), equalTo(expected)); + assertThat(BigQueryAvroUtils.formatTimestamp(micros), equalTo(expected + " UTC")); } @Test - public void testFormatTimestampLeadingZeroesOnMicros() { - assertThat( - BigQueryAvroUtils.formatTimestamp(1452062291000456L), - equalTo("2016-01-06 06:38:11.000456 UTC")); + public void testFormatTimestampMillis() { + long millis = 1452062291123L; + long micros = millis * 1000L; + String expected = "2016-01-06 06:38:11.123"; + assertThat(BigQueryAvroUtils.formatDatetime(micros), equalTo(expected)); + assertThat(BigQueryAvroUtils.formatTimestamp(micros), equalTo(expected + " UTC")); } @Test - public void testFormatTimestampTrailingZeroesOnMicros() { - assertThat( - BigQueryAvroUtils.formatTimestamp(1452062291123000L), - equalTo("2016-01-06 06:38:11.123000 UTC")); + public void testFormatTimestampSeconds() { + long seconds = 1452062291L; + long micros = seconds * 1000L * 1000L; + String expected = "2016-01-06 06:38:11"; + assertThat(BigQueryAvroUtils.formatDatetime(micros), equalTo(expected)); + assertThat(BigQueryAvroUtils.formatTimestamp(micros), equalTo(expected + " UTC")); } @Test public void testFormatTimestampNegative() { - assertThat(BigQueryAvroUtils.formatTimestamp(-1L), equalTo("1969-12-31 23:59:59.999999 UTC")); - assertThat( - BigQueryAvroUtils.formatTimestamp(-100_000L), equalTo("1969-12-31 23:59:59.900000 UTC")); - assertThat(BigQueryAvroUtils.formatTimestamp(-1_000_000L), equalTo("1969-12-31 23:59:59 UTC")); + assertThat(BigQueryAvroUtils.formatDatetime(-1L), equalTo("1969-12-31 23:59:59.999999")); + assertThat(BigQueryAvroUtils.formatDatetime(-100_000L), equalTo("1969-12-31 23:59:59.900")); + assertThat(BigQueryAvroUtils.formatDatetime(-1_000_000L), equalTo("1969-12-31 23:59:59")); // No leap seconds before 1972. 477 leap years from 1 through 1969. assertThat( - BigQueryAvroUtils.formatTimestamp(-(1969L * 365 + 477) * 86400 * 1_000_000), - equalTo("0001-01-01 00:00:00 UTC")); + BigQueryAvroUtils.formatDatetime(-(1969L * 365 + 477) * 86400 * 1_000_000), + equalTo("0001-01-01 00:00:00")); } @Test @@ -501,48 +816,4 @@ public void testSchemaCollisionsInAvroConversion() { String output = BigQueryAvroUtils.toGenericAvroSchema(schema, false).toString(); assertThat(output.length(), greaterThan(0)); } - - /** Pojo class used as the record type in tests. */ - @SuppressWarnings("unused") // Used by Avro reflection. - static class Bird { - long number; - @Nullable String species; - @Nullable Double quality; - @Nullable Long quantity; - - @AvroSchema(value = "[\"null\", {\"type\": \"long\", \"logicalType\": \"timestamp-micros\"}]") - Instant birthday; - - @AvroSchema( - value = - "[\"null\", {\"type\": \"bytes\", \"logicalType\": \"decimal\", \"precision\": 38, \"scale\": 9}]") - BigDecimal birthdayMoney; - - @AvroSchema( - value = - "[\"null\", {\"type\": \"bytes\", \"logicalType\": \"decimal\", \"precision\": 77, \"scale\": 38}]") - BigDecimal lotteryWinnings; - - @AvroSchema(value = "[\"null\", {\"type\": \"string\", \"sqlType\": \"GEOGRAPHY\"}]") - String geoPositions; - - @Nullable Boolean flighted; - @Nullable ByteBuffer sound; - @Nullable Utf8 anniversaryDate; - @Nullable String anniversaryDatetime; - @Nullable Utf8 anniversaryTime; - @Nullable SubBird scion; - SubBird[] associates; - - static class SubBird { - @Nullable String species; - - public SubBird() {} - } - - public Bird() { - associates = new SubBird[1]; - associates[0] = new SubBird(); - } - } } From 76bcec0086cf0137d2ea6c6259eadc15456e96bc Mon Sep 17 00:00:00 2001 From: Damon Date: Tue, 10 Dec 2024 09:50:51 -0800 Subject: [PATCH 10/33] Remove the use of google-github-actions/auth@v1 (#33338) * Remove use of google-github-actions/auth step. * Create beam_PostCommit_Java_IO_Performance_Tests.json --- .../beam_PostCommit_Java_IO_Performance_Tests.json | 4 ++++ .../workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 5 ----- 2 files changed, 4 insertions(+), 5 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Java_IO_Performance_Tests.json diff --git a/.github/trigger_files/beam_PostCommit_Java_IO_Performance_Tests.json b/.github/trigger_files/beam_PostCommit_Java_IO_Performance_Tests.json new file mode 100644 index 000000000000..b26833333238 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Java_IO_Performance_Tests.json @@ -0,0 +1,4 @@ +{ + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 2 +} diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index a6a2749c8d82..c6d9dc2236d3 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -88,11 +88,6 @@ jobs: uses: ./.github/actions/setup-environment-action with: java-version: default - - name: Authenticate on GCP - uses: google-github-actions/auth@v1 - with: - credentials_json: ${{ secrets.GCP_SA_KEY }} - project_id: ${{ secrets.GCP_PROJECT_ID }} - name: run scheduled javaPostcommitIOPerformanceTests script if: github.event_name == 'schedule' #This ensures only scheduled runs publish metrics publicly by changing which exportTable is configured uses: ./.github/actions/gradle-command-self-hosted-action From b4fbf8957e9a39ec018ace9aec284a7af3144f2d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 10 Dec 2024 17:14:43 -0500 Subject: [PATCH 11/33] Add hadoop-auth (#33342) * add hadoop auth * trigger xlang tests * place dep in expansion service --- .../trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json | 2 +- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 1 + sdks/java/io/expansion-service/build.gradle | 1 + sdks/java/io/iceberg/build.gradle | 3 +-- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json index c537844dc84a..b26833333238 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 3 + "modification": 2 } diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 2abd43a5d4cc..a59c1d7630b0 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -795,6 +795,7 @@ class BeamModulePlugin implements Plugin { grpc_xds : "io.grpc:grpc-xds", // google_cloud_platform_libraries_bom sets version guava : "com.google.guava:guava:$guava_version", guava_testlib : "com.google.guava:guava-testlib:$guava_version", + hadoop_auth : "org.apache.hadoop:hadoop-auth:$hadoop_version", hadoop_client : "org.apache.hadoop:hadoop-client:$hadoop_version", hadoop_common : "org.apache.hadoop:hadoop-common:$hadoop_version", hadoop_mapreduce_client_core : "org.apache.hadoop:hadoop-mapreduce-client-core:$hadoop_version", diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index cc8eccf98997..421719b8f986 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -54,6 +54,7 @@ dependencies { permitUnusedDeclared project(":sdks:java:io:kafka:upgrade") // BEAM-11761 // **** IcebergIO runtime dependencies **** + runtimeOnly library.java.hadoop_auth runtimeOnly library.java.hadoop_client // Needed when using GCS as the warehouse location. runtimeOnly library.java.bigdataoss_gcs_connector diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index fa1e2426ce69..0cfa8da4eb7d 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -53,15 +53,14 @@ dependencies { implementation "org.apache.iceberg:iceberg-api:$iceberg_version" implementation "org.apache.iceberg:iceberg-parquet:$iceberg_version" implementation "org.apache.iceberg:iceberg-orc:$iceberg_version" - runtimeOnly "org.apache.iceberg:iceberg-gcp:$iceberg_version" implementation library.java.hadoop_common + runtimeOnly "org.apache.iceberg:iceberg-gcp:$iceberg_version" testImplementation project(":sdks:java:managed") testImplementation library.java.hadoop_client testImplementation library.java.bigdataoss_gcsio testImplementation library.java.bigdataoss_gcs_connector testImplementation library.java.bigdataoss_util_hadoop - testImplementation "org.apache.iceberg:iceberg-gcp:$iceberg_version" testImplementation "org.apache.iceberg:iceberg-data:$iceberg_version" testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(":sdks:java:extensions:google-cloud-platform-core") From 7a94b6192332c49e9a39fe6bb316f4c3bec62929 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Dec 2024 16:43:29 -0800 Subject: [PATCH 12/33] Bump golang.org/x/net from 0.31.0 to 0.32.0 in /sdks (#33326) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.31.0 to 0.32.0. - [Commits](https://github.com/golang/net/compare/v0.31.0...v0.32.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 6 +++--- sdks/go.sum | 16 ++++++++-------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index a73cd530325f..0a09ee59c805 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,10 +53,10 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.17.1 - golang.org/x/net v0.31.0 + golang.org/x/net v0.32.0 golang.org/x/oauth2 v0.24.0 golang.org/x/sync v0.10.0 - golang.org/x/sys v0.27.0 + golang.org/x/sys v0.28.0 golang.org/x/text v0.21.0 google.golang.org/api v0.210.0 google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 @@ -190,7 +190,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.29.0 // indirect + golang.org/x/crypto v0.30.0 // indirect golang.org/x/mod v0.20.0 // indirect golang.org/x/tools v0.24.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect diff --git a/sdks/go.sum b/sdks/go.sum index a0f784b9789e..bf2a9478a787 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1266,8 +1266,8 @@ golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.29.0 h1:L5SG1JTTXupVV3n6sUqMTeWbjAyfPwoda2DLX8J8FrQ= -golang.org/x/crypto v0.29.0/go.mod h1:+F4F4N5hv6v38hfeYwTdx20oUvLLc+QfrE9Ax9HtgRg= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1388,8 +1388,8 @@ golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= -golang.org/x/net v0.31.0 h1:68CPQngjLL0r2AlUKiSxtQFKvzRVbnzLwMUn5SzcLHo= -golang.org/x/net v0.31.0/go.mod h1:P4fl1q7dY2hnZFxEk4pPSkDHF+QqjitcnDjUQyMM+pM= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1526,8 +1526,8 @@ golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.27.0 h1:wBqf8DvsY9Y/2P8gAfPDEYNuS30J4lPHJxXSb/nJZ+s= -golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= @@ -1536,8 +1536,8 @@ golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= -golang.org/x/term v0.26.0 h1:WEQa6V3Gja/BhNxg540hBip/kkaYtRg3cxg4oXSw4AU= -golang.org/x/term v0.26.0/go.mod h1:Si5m1o57C5nBNQo5z1iq+XDijt21BDBDp2bK0QI8e3E= +golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From 04b059ff2e40dfd1ef295e35610f8db4a5ec4613 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Dec 2024 22:32:26 -0500 Subject: [PATCH 13/33] Remove mandatory beam-sdks-io-kafka dependency for dataflow worker jar (#33302) --- .../worker/build.gradle | 1 - ...icsToPerStepNamespaceMetricsConverter.java | 6 ++-- .../worker/StreamingDataflowWorker.java | 5 --- .../dataflow/worker/streaming/StageInfo.java | 5 ++- .../beam/sdk/io/kafka/KafkaIOInitializer.java | 34 +++++++++++++++++++ 5 files changed, 40 insertions(+), 11 deletions(-) create mode 100644 sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOInitializer.java diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index 92beccd067e2..b7e6e981effe 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -54,7 +54,6 @@ def sdk_provided_project_dependencies = [ ":runners:google-cloud-dataflow-java", ":sdks:java:extensions:avro", ":sdks:java:extensions:google-cloud-platform-core", - ":sdks:java:io:kafka", // For metric propagation into worker ":sdks:java:io:google-cloud-platform", ] diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java index 77f867793ae2..91baefa0be4c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java @@ -32,7 +32,6 @@ import java.util.Map.Entry; import java.util.Optional; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; -import org.apache.beam.sdk.io.kafka.KafkaSinkMetrics; import org.apache.beam.sdk.metrics.LabeledMetricNameUtils; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.util.HistogramData; @@ -43,6 +42,9 @@ * converter. */ public class MetricsToPerStepNamespaceMetricsConverter { + // Avoids to introduce mandatory kafka-io dependency to Dataflow worker + // keep in sync with org.apache.beam.sdk.io.kafka.KafkaSinkMetrics.METRICS_NAMESPACE + public static String KAFKA_SINK_METRICS_NAMESPACE = "KafkaSink"; private static Optional getParsedMetricName( MetricName metricName, @@ -70,7 +72,7 @@ private static Optional convertCounterToMetricValue( if (value == 0 || (!metricName.getNamespace().equals(BigQuerySinkMetrics.METRICS_NAMESPACE) - && !metricName.getNamespace().equals(KafkaSinkMetrics.METRICS_NAMESPACE))) { + && !metricName.getNamespace().equals(KAFKA_SINK_METRICS_NAMESPACE))) { return Optional.empty(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 088a28e9b2db..0112ab4af80a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -110,7 +110,6 @@ import org.apache.beam.sdk.fn.JvmInitializers; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; -import org.apache.beam.sdk.io.kafka.KafkaSinkMetrics; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -835,10 +834,6 @@ public static void main(String[] args) throws Exception { enableBigQueryMetrics(); } - if (DataflowRunner.hasExperiment(options, "enable_kafka_metrics")) { - KafkaSinkMetrics.setSupportKafkaMetrics(true); - } - JvmInitializers.runBeforeProcessing(options); worker.startStatusPages(); worker.start(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java index 525464ef2e1f..d9fe95f3421b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker.streaming; +import static org.apache.beam.runners.dataflow.worker.MetricsToPerStepNamespaceMetricsConverter.KAFKA_SINK_METRICS_NAMESPACE; import static org.apache.beam.sdk.metrics.Metrics.THROTTLE_TIME_COUNTER_NAME; import com.google.api.services.dataflow.model.CounterStructuredName; @@ -35,7 +36,6 @@ import org.apache.beam.runners.dataflow.worker.counters.DataflowCounterUpdateExtractor; import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; -import org.apache.beam.sdk.io.kafka.KafkaSinkMetrics; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** Contains a few of the stage specific fields. E.g. metrics container registry, counters etc. */ @@ -120,8 +120,7 @@ private void translateKnownPerWorkerCounters(List metri for (PerStepNamespaceMetrics perStepnamespaceMetrics : metrics) { if (!BigQuerySinkMetrics.METRICS_NAMESPACE.equals( perStepnamespaceMetrics.getMetricsNamespace()) - && !KafkaSinkMetrics.METRICS_NAMESPACE.equals( - perStepnamespaceMetrics.getMetricsNamespace())) { + && !KAFKA_SINK_METRICS_NAMESPACE.equals(perStepnamespaceMetrics.getMetricsNamespace())) { continue; } for (MetricValue metric : perStepnamespaceMetrics.getMetricValues()) { diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOInitializer.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOInitializer.java new file mode 100644 index 000000000000..3dfb31715ced --- /dev/null +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOInitializer.java @@ -0,0 +1,34 @@ +/* + * 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. + */ +package org.apache.beam.sdk.io.kafka; + +import com.google.auto.service.AutoService; +import org.apache.beam.sdk.harness.JvmInitializer; +import org.apache.beam.sdk.options.ExperimentalOptions; +import org.apache.beam.sdk.options.PipelineOptions; + +/** Initialize KafkaIO feature flags on worker. */ +@AutoService(JvmInitializer.class) +public class KafkaIOInitializer implements JvmInitializer { + @Override + public void beforeProcessing(PipelineOptions options) { + if (ExperimentalOptions.hasExperiment(options, "enable_kafka_metrics")) { + KafkaSinkMetrics.setSupportKafkaMetrics(true); + } + } +} From b26e2fad761bc85832192dfa0abf335ae5c31111 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Dec 2024 07:51:05 -0800 Subject: [PATCH 14/33] Bump github.com/nats-io/nats-server/v2 from 2.10.22 to 2.10.23 in /sdks (#33352) Bumps [github.com/nats-io/nats-server/v2](https://github.com/nats-io/nats-server) from 2.10.22 to 2.10.23. - [Release notes](https://github.com/nats-io/nats-server/releases) - [Changelog](https://github.com/nats-io/nats-server/blob/main/.goreleaser.yml) - [Commits](https://github.com/nats-io/nats-server/compare/v2.10.22...v2.10.23) --- updated-dependencies: - dependency-name: github.com/nats-io/nats-server/v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 0a09ee59c805..5406c2b70cbc 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -44,7 +44,7 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.0 - github.com/nats-io/nats-server/v2 v2.10.22 + github.com/nats-io/nats-server/v2 v2.10.23 github.com/nats-io/nats.go v1.37.0 github.com/proullon/ramsql v0.1.4 github.com/spf13/cobra v1.8.1 @@ -98,7 +98,7 @@ require ( github.com/moby/sys/user v0.1.0 // indirect github.com/moby/sys/userns v0.1.0 // indirect github.com/nats-io/jwt/v2 v2.5.8 // indirect - github.com/nats-io/nkeys v0.4.7 // indirect + github.com/nats-io/nkeys v0.4.8 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect diff --git a/sdks/go.sum b/sdks/go.sum index bf2a9478a787..bb96c54af087 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1085,12 +1085,12 @@ github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/nats-io/jwt/v2 v2.5.8 h1:uvdSzwWiEGWGXf+0Q+70qv6AQdvcvxrv9hPM0RiPamE= github.com/nats-io/jwt/v2 v2.5.8/go.mod h1:ZdWS1nZa6WMZfFwwgpEaqBV8EPGVgOTDHN/wTbz0Y5A= -github.com/nats-io/nats-server/v2 v2.10.22 h1:Yt63BGu2c3DdMoBZNcR6pjGQwk/asrKU7VX846ibxDA= -github.com/nats-io/nats-server/v2 v2.10.22/go.mod h1:X/m1ye9NYansUXYFrbcDwUi/blHkrgHh2rgCJaakonk= +github.com/nats-io/nats-server/v2 v2.10.23 h1:jvfb9cEi5h8UG6HkZgJGdn9f1UPaX3Dohk0PohEekJI= +github.com/nats-io/nats-server/v2 v2.10.23/go.mod h1:hMFnpDT2XUXsvHglABlFl/uroQCCOcW6X/0esW6GpBk= github.com/nats-io/nats.go v1.37.0 h1:07rauXbVnnJvv1gfIyghFEo6lUcYRY0WXc3x7x0vUxE= github.com/nats-io/nats.go v1.37.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= -github.com/nats-io/nkeys v0.4.7 h1:RwNJbbIdYCoClSDNY7QVKZlyb/wfT6ugvFCiKy6vDvI= -github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDmGD0nc= +github.com/nats-io/nkeys v0.4.8 h1:+wee30071y3vCZAYRsnrmIPaOe47A/SkK/UBDPdIV70= +github.com/nats-io/nkeys v0.4.8/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDmGD0nc= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/ncw/swift v1.0.52/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= From 00ec6cc2d1a24af27defc3f19c77c51cda637104 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rados=C5=82aw=20Stankiewicz?= Date: Wed, 11 Dec 2024 17:51:20 +0100 Subject: [PATCH 15/33] fix Error processing result file: CData section too big found in publish-int-test-result-action #29966 (#33353) --- .github/workflows/IO_Iceberg_Unit_Tests.yml | 1 + .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 3 ++- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 3 ++- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 3 ++- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 3 ++- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 3 ++- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 3 ++- .github/workflows/beam_PostCommit_Java.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 3 ++- .../workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 1 + .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 3 ++- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml | 3 ++- .../workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml | 1 + .../workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml | 3 ++- .../workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 3 ++- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 1 + .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 3 ++- .../workflows/beam_PostCommit_Java_IO_Performance_Tests.yml | 1 + .../workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 3 ++- .../workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 3 ++- .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 3 ++- .github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 1 + .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 3 ++- .../workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 3 ++- ...m_PostCommit_Java_ValidatesDistrolessContainer_Dataflow.yml | 1 + .../beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 3 ++- ...m_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 3 ++- ...beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 3 ++- ...m_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 3 ++- .../workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml | 3 ++- ...eam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 3 ++- .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 1 + .../beam_PostCommit_Java_ValidatesRunner_Flink_Java8.yml | 1 + .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 3 ++- .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 3 ++- ...ostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java8.yml | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 3 ++- .github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 3 ++- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 3 ++- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 3 ++- .github/workflows/beam_PostCommit_Python.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Arm.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Dependency.yml | 1 + .github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 1 + .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 3 ++- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 3 ++- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 1 + ...m_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 3 ++- ...PostCommit_Python_ValidatesDistrolessContainer_Dataflow.yml | 1 + .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 3 ++- .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 3 ++- .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 3 ++- .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 3 ++- .../workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 3 ++- .github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml | 3 ++- .github/workflows/beam_PostCommit_SQL.yml | 1 + .github/workflows/beam_PostCommit_TransformService_Direct.yml | 3 ++- .github/workflows/beam_PostCommit_XVR_Direct.yml | 3 ++- .github/workflows/beam_PostCommit_XVR_Flink.yml | 1 + .github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 1 + .../workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 3 ++- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 3 ++- .../workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 3 ++- .github/workflows/beam_PostCommit_XVR_Samza.yml | 3 ++- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 3 ++- .github/workflows/beam_PreCommit_ItFramework.yml | 3 ++- .github/workflows/beam_PreCommit_Java.yml | 1 + .../beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml | 1 + .../beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml | 1 + .../workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml | 3 ++- .../workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml | 1 + .../beam_PreCommit_Java_File-schema-transform_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Flink_Versions.yml | 3 ++- .github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_IOs_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml | 1 + .github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml | 1 + .../beam_PreCommit_Java_RequestResponse_IO_Direct.yml | 1 + .../workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Spark3_Versions.yml | 3 ++- .github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml | 1 + .github/workflows/beam_PreCommit_Python.yml | 3 ++- .github/workflows/beam_PreCommit_Python_Coverage.yml | 3 ++- .github/workflows/beam_PreCommit_Python_Dataframes.yml | 3 ++- .github/workflows/beam_PreCommit_Python_Examples.yml | 3 ++- .github/workflows/beam_PreCommit_Python_Integration.yml | 3 ++- .github/workflows/beam_PreCommit_Python_ML.yml | 3 ++- .github/workflows/beam_PreCommit_Python_PVR_Flink.yml | 3 ++- .github/workflows/beam_PreCommit_Python_Runners.yml | 3 ++- .github/workflows/beam_PreCommit_Python_Transforms.yml | 3 ++- .github/workflows/beam_PreCommit_SQL.yml | 1 + .github/workflows/beam_PreCommit_SQL_Java17.yml | 1 + .github/workflows/beam_PreCommit_SQL_Java8.yml | 1 + .github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml | 1 + 136 files changed, 212 insertions(+), 76 deletions(-) diff --git a/.github/workflows/IO_Iceberg_Unit_Tests.yml b/.github/workflows/IO_Iceberg_Unit_Tests.yml index 0d72b0da8597..d063f6ac71db 100644 --- a/.github/workflows/IO_Iceberg_Unit_Tests.yml +++ b/.github/workflows/IO_Iceberg_Unit_Tests.yml @@ -111,6 +111,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 2b631d2f7664..659c85b002df 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -124,4 +124,5 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index af0569f4784a..74932079fe4c 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -102,4 +102,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 9e3962e2576e..05e5369a6384 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -102,4 +102,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index 7514bd5cacb3..32db2cff6cbc 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -102,4 +102,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 6ac07a1bd76c..d04a6e63c800 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -101,4 +101,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index e9ef9cd1716a..756ecb5a58c2 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -115,4 +115,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 3428551cb8f9..4fafa3b2a993 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index e3a9db23ed67..8ffcc4a28a71 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 1a6f7c14db50..8707b515e10b 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -110,3 +110,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index e7c2aa6fe7e2..752b15936b5f 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -94,4 +94,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 3c0a46d6bb40..cb107572b621 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml index 469d7e31f173..81725c4005af 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow.yml @@ -89,4 +89,5 @@ jobs: uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 9fd84daef63b..eacdfe5a5c23 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -119,3 +119,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index 13ab05f8f173..efb926681cbf 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -97,4 +97,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 9be8a34f3732..1882cdf1d76b 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -91,4 +91,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index cd2486ae8e10..05b28ac93658 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -104,4 +104,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index ca06e72877c7..a746acb4333f 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -92,4 +92,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index e42d6a88b8df..f72910bd15bc 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -94,3 +94,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index 8008daf4584f..c3620e46fac9 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -92,4 +92,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index 67a48b105955..1202ecc0e27f 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -100,4 +100,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml index c6d9dc2236d3..6023a895a458 100644 --- a/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml +++ b/.github/workflows/beam_PostCommit_Java_IO_Performance_Tests.yml @@ -117,3 +117,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 37f784770477..323f85b9851a 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -91,4 +91,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 377602ad08dd..1ccb26f5aa1f 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -96,4 +96,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 80406cf4eb0c..02ac93135957 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -91,4 +91,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index 3cbc317317c2..2cbf60a48d2e 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -96,4 +96,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml index 97fd1fb4913e..6a7058ef566d 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java21.yml @@ -97,4 +97,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index 1a7405836f69..1559061634d3 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -91,4 +91,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index eec4867a997b..1b4f8c5bcce5 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -91,4 +91,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 987be7789b29..8c5fcb1acff4 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -91,3 +91,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 7cc48ebd4b0e..c1a22b9c871d 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -100,4 +100,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index ad10bfc684d8..76ab560f15ec 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesDistrolessContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesDistrolessContainer_Dataflow.yml index 4fb236c7c991..73fd6f0b78fa 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesDistrolessContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesDistrolessContainer_Dataflow.yml @@ -113,3 +113,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index d66381393725..c85c0b8468dc 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index da2ba2f88465..5963a33007e0 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -111,4 +111,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index edb055321c87..2e8227fb84a6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 8957ce7de053..2abc081e6ae5 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index 2a98746a0b84..fde10e0898e9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 3f48bb921805..f439be9ec58e 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 75ebbda93f80..eb70a654c93d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -106,4 +106,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index f79ca8747828..1442f5ffafc0 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -93,3 +93,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.yml index c51c39987236..0f12ce6f90ef 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java8.yml @@ -110,3 +110,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 794308d3a85e..edcb45303fd4 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -96,4 +96,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index d1f264aaac01..d05963263931 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 15863d4c8c9b..da04582a7caa 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java8.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java8.yml index c05284186617..8d531c120dd6 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java8.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java8.yml @@ -108,4 +108,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 522cb300c687..8310e5ed8bb2 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -90,4 +90,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 36fc06aea421..3b130b6d290f 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -89,4 +89,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 37bfe68d9b20..318b5104c39c 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -94,4 +94,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index ce7be60133d7..0712dfb255b7 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -94,4 +94,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 4770515c75fb..93b85a318487 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 48fb00b1bb9d..352b95e6747a 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -124,4 +124,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Dependency.yml b/.github/workflows/beam_PostCommit_Python_Dependency.yml index 6e7c4ddbd3eb..80e1bbc290c9 100644 --- a/.github/workflows/beam_PostCommit_Python_Dependency.yml +++ b/.github/workflows/beam_PostCommit_Python_Dependency.yml @@ -96,3 +96,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 4ce3b1893215..bf8330a2ae58 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -94,4 +94,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index a6bb49f4e444..e271b7da9a7b 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -101,4 +101,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index f23674a2c70a..28fd13c181b3 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -102,3 +102,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index d866d412507b..5df6bcf8c01c 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -101,4 +101,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 578775a9d3ed..0d334b679dc5 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index bcd936324124..6e16f43476b2 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -108,3 +108,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index f2eba045722c..3ab7257f8a9d 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -106,4 +106,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesDistrolessContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesDistrolessContainer_Dataflow.yml index 6f8a7bdd0631..c294dd3c9068 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesDistrolessContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesDistrolessContainer_Dataflow.yml @@ -118,3 +118,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 1876950c7a93..f8daa1a96634 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index f837c7476e12..9277bd68fc01 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -103,4 +103,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index 91c249adf338..e058724cd2ac 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -102,4 +102,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index 7e87aaff22cc..a47f758ed410 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -101,4 +101,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index b3f37c6b39f0..bd266cf6fdab 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index 137d7bc13d2f..6d26d1c46012 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -92,4 +92,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 8fc0db189078..08e99fa0fe0f 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -95,4 +95,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml index 5092a1981154..a7643c795af4 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Direct.yml @@ -93,4 +93,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index c7d0b6dc98b9..aebea2b0564b 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -91,3 +91,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index cb339eb9fb40..d0d72f3df13c 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -98,4 +98,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index 023ae4f8cd31..af8b7fb1bf54 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 1f1d7d863b7e..fe4404247448 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -111,3 +111,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 228f10b90cd0..0620023ce7d2 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -102,3 +102,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 66770c9a1683..11a8a5c5f4f7 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -95,4 +95,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index bfb602f89daf..c393a4113589 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -92,4 +92,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index f1269a0ddd09..082aeb3f2ab2 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -95,4 +95,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 2d26c9131839..7e2dca61d41d 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -111,4 +111,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index c1880e01292b..17fb58d9dd73 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_ItFramework.yml b/.github/workflows/beam_PreCommit_ItFramework.yml index e078d4645757..e803fc023c67 100644 --- a/.github/workflows/beam_PreCommit_ItFramework.yml +++ b/.github/workflows/beam_PreCommit_ItFramework.yml @@ -101,4 +101,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java.yml b/.github/workflows/beam_PreCommit_Java.yml index 20dafca72a57..bc25fb94f8f0 100644 --- a/.github/workflows/beam_PreCommit_Java.yml +++ b/.github/workflows/beam_PreCommit_Java.yml @@ -198,6 +198,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml index ecbc85ca1b1d..cf0d0b660782 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services2_IO_Direct.yml @@ -130,6 +130,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml index 55935251e6d9..9053bb730371 100644 --- a/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Amazon-Web-Services_IO_Direct.yml @@ -130,6 +130,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml index 4fbacecde4a4..8c0bb07e1acb 100644 --- a/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Azure_IO_Direct.yml @@ -123,6 +123,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml index e37bc5c56e2e..317b2e1f2ec1 100644 --- a/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cassandra_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml index 68ebe3c28fb3..3e0208b758cc 100644 --- a/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Cdap_IO_Direct.yml @@ -109,6 +109,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml index 5c0b169b0ba1..2be7607b5bc7 100644 --- a/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Clickhouse_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml index ce91551c1121..6901e56c0bbb 100644 --- a/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Csv_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml index b6a0e6b999bd..6f32c3844b1a 100644 --- a/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Debezium_IO_Direct.yml @@ -114,6 +114,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml index 78ab882d4774..11a95cf476c7 100644 --- a/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_ElasticSearch_IO_Direct.yml @@ -117,6 +117,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml index 4bfb20a28e7c..8e22318bdcb9 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow.yml @@ -117,4 +117,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml index 72fc945018f6..763de153b137 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java21.yml @@ -133,6 +133,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 with: diff --git a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml index e96dc7c883bf..e121fe1e53a2 100644 --- a/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_File-schema-transform_IO_Direct.yml @@ -106,6 +106,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml index 19b0d56a8051..09bf906e5a38 100644 --- a/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Flink_Versions.yml @@ -104,4 +104,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml index 2256d0a91cb8..ee5bea3d3ab3 100644 --- a/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_GCP_IO_Direct.yml @@ -127,6 +127,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml index c481251bef03..0e6bd11e7f1e 100644 --- a/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Google-ads_IO_Direct.yml @@ -103,6 +103,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml index 3b99e30bfbac..c334edd7f32d 100644 --- a/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HBase_IO_Direct.yml @@ -107,6 +107,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml index 6d45ba82aa49..ed079c1e9dd1 100644 --- a/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_HCatalog_IO_Direct.yml @@ -122,6 +122,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml index c2beaa3c1099..442085586a3c 100644 --- a/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Hadoop_IO_Direct.yml @@ -145,6 +145,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml index 4e19a56dde0c..cd73d402c7ea 100644 --- a/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_IOs_Direct.yml @@ -122,6 +122,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml index 903a7cd73526..977781de506f 100644 --- a/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_InfluxDb_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml index 071cdb3bda3e..4759d48d979f 100644 --- a/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_JDBC_IO_Direct.yml @@ -112,6 +112,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml index 650036345274..935315463358 100644 --- a/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Jms_IO_Direct.yml @@ -112,6 +112,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 0ede01376ce7..f177ec85fada 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -114,6 +114,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml index 494a738abf45..785748e793e9 100644 --- a/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kinesis_IO_Direct.yml @@ -137,6 +137,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml index e38c9a761dee..853e52db14db 100644 --- a/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kudu_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml index 11be57c05759..b3292ac5f29b 100644 --- a/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_MongoDb_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml index ac8800f55cdf..ed0189d8006b 100644 --- a/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Mqtt_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml index 553300f1889c..62429a611f2a 100644 --- a/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Neo4j_IO_Direct.yml @@ -114,6 +114,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml index 5feb0270c68c..48f165f4e59f 100644 --- a/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml +++ b/.github/workflows/beam_PreCommit_Java_PVR_Flink_Docker.yml @@ -115,3 +115,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true diff --git a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml index 0bec073fc37b..d217f0e88c39 100644 --- a/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Parquet_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index e25b4ff6fa94..3a9d62fb64c6 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -123,6 +123,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml index eb343f193395..c72b04bc108d 100644 --- a/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RabbitMq_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml index 13b9c26b4b81..cd4ddc387ffc 100644 --- a/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Redis_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml index f1e8c3699aa6..1037ab972447 100644 --- a/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_RequestResponse_IO_Direct.yml @@ -103,6 +103,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml index 4d289882353e..478dad9989b9 100644 --- a/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_SingleStore_IO_Direct.yml @@ -107,6 +107,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml index 03577eff2860..403c26ac0ab0 100644 --- a/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Snowflake_IO_Direct.yml @@ -116,6 +116,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml index 5aeaaec11dec..ca05b44875cb 100644 --- a/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml @@ -112,6 +112,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml index e6138a0c10d9..80cd5e492992 100644 --- a/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solr_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml index 18f5a6c0c86e..c6b2d7e57128 100644 --- a/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml +++ b/.github/workflows/beam_PreCommit_Java_Spark3_Versions.yml @@ -112,4 +112,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/build/test-results/**/*.xml' \ No newline at end of file + files: '**/build/test-results/**/*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml index 73a1a0b5cdb2..53f3c4327739 100644 --- a/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Splunk_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml index 4cddfa728cc1..b5336537c556 100644 --- a/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Thrift_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml index e08b5048b359..195e9aa1f168 100644 --- a/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Tika_IO_Direct.yml @@ -105,6 +105,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index fb1c6c80873a..68c69ae953a4 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 0e295250817d..3c7c3b05d8bc 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -104,4 +104,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Dataframes.yml b/.github/workflows/beam_PreCommit_Python_Dataframes.yml index f045842e061d..ecbb1a30e5f7 100644 --- a/.github/workflows/beam_PreCommit_Python_Dataframes.yml +++ b/.github/workflows/beam_PreCommit_Python_Dataframes.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index 09d46217d6d6..44329f63014d 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Integration.yml b/.github/workflows/beam_PreCommit_Python_Integration.yml index 20aade431f6d..3a709c70f077 100644 --- a/.github/workflows/beam_PreCommit_Python_Integration.yml +++ b/.github/workflows/beam_PreCommit_Python_Integration.yml @@ -116,4 +116,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index 714eceef5f6b..3b3a2150ac28 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml index dbc1264fcc04..5dd12d49ccd9 100644 --- a/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml +++ b/.github/workflows/beam_PreCommit_Python_PVR_Flink.yml @@ -125,4 +125,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Runners.yml b/.github/workflows/beam_PreCommit_Python_Runners.yml index 5db6e94be781..689d9b2c3c3f 100644 --- a/.github/workflows/beam_PreCommit_Python_Runners.yml +++ b/.github/workflows/beam_PreCommit_Python_Runners.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 820ca3e26df6..431b82c02fb7 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -109,4 +109,5 @@ jobs: with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} - files: '**/pytest*.xml' \ No newline at end of file + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_SQL.yml b/.github/workflows/beam_PreCommit_SQL.yml index b4002fcc2a79..5bc8bb581955 100644 --- a/.github/workflows/beam_PreCommit_SQL.yml +++ b/.github/workflows/beam_PreCommit_SQL.yml @@ -103,6 +103,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 0e5dcc87d16f..1cfd7502389d 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -110,6 +110,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_SQL_Java8.yml b/.github/workflows/beam_PreCommit_SQL_Java8.yml index 23938821b2e8..6b59739dd72d 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java8.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java8.yml @@ -114,6 +114,7 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' + large_files: true - name: Archive SpotBugs Results uses: actions/upload-artifact@v4 if: always() diff --git a/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml b/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml index b17913946a7e..b9e310a7a133 100644 --- a/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml +++ b/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml @@ -105,3 +105,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' + large_files: true From e2ff659c98c245df7fa8b833b297b8c75895f804 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Wed, 11 Dec 2024 17:11:05 +0000 Subject: [PATCH 16/33] Reapply "SolaceIO.Read: handle occasional cases when finalizeCheckpoint is not executed (#32962)" (#33259) (#33269) This reverts commit 4356253a5e8124bf39152a9ead9ad26ef7267750. --- .../broker/BasicAuthJcsmpSessionService.java | 18 +-- .../sdk/io/solace/broker/MessageReceiver.java | 7 - .../sdk/io/solace/broker/SessionService.java | 7 - .../solace/broker/SolaceMessageReceiver.java | 17 +-- .../io/solace/read/SolaceCheckpointMark.java | 46 +++--- .../io/solace/read/UnboundedSolaceReader.java | 135 +++++++++++++----- .../io/solace/MockEmptySessionService.java | 5 - .../sdk/io/solace/MockSessionService.java | 10 -- .../beam/sdk/io/solace/SolaceIOReadTest.java | 20 +-- 9 files changed, 138 insertions(+), 127 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java index b2196dbf1067..d4c9a3ec6210 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java @@ -102,10 +102,7 @@ public void close() { if (messageReceiver != null) { messageReceiver.close(); } - if (messageProducer != null) { - messageProducer.close(); - } - if (!isClosed()) { + if (jcsmpSession != null) { checkStateNotNull(jcsmpSession).closeSession(); } return 0; @@ -119,8 +116,9 @@ public MessageReceiver getReceiver() { this.messageReceiver = retryCallableManager.retryCallable( this::createFlowReceiver, ImmutableSet.of(JCSMPException.class)); + this.messageReceiver.start(); } - return this.messageReceiver; + return checkStateNotNull(this.messageReceiver); } @Override @@ -138,15 +136,10 @@ public java.util.Queue getPublishedResultsQueue() { return publishedResultsQueue; } - @Override - public boolean isClosed() { - return jcsmpSession == null || jcsmpSession.isClosed(); - } - private MessageProducer createXMLMessageProducer(SubmissionMode submissionMode) throws JCSMPException, IOException { - if (isClosed()) { + if (jcsmpSession == null) { connectWriteSession(submissionMode); } @@ -165,9 +158,6 @@ private MessageProducer createXMLMessageProducer(SubmissionMode submissionMode) } private MessageReceiver createFlowReceiver() throws JCSMPException, IOException { - if (isClosed()) { - connectSession(); - } Queue queue = JCSMPFactory.onlyInstance() diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java index 95f989bd1be9..017a63260678 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java @@ -35,13 +35,6 @@ public interface MessageReceiver { */ void start(); - /** - * Returns {@literal true} if the message receiver is closed, {@literal false} otherwise. - * - *

A message receiver is closed when it is no longer able to receive messages. - */ - boolean isClosed(); - /** * Receives a message from the broker. * diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java index 84a876a9d0bc..6dcd0b652616 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java @@ -120,13 +120,6 @@ public abstract class SessionService implements Serializable { /** Gracefully closes the connection to the service. */ public abstract void close(); - /** - * Checks whether the connection to the service is currently closed. This method is called when an - * `UnboundedSolaceReader` is starting to read messages - a session will be created if this - * returns true. - */ - public abstract boolean isClosed(); - /** * Returns a MessageReceiver object for receiving messages from Solace. If it is the first time * this method is used, the receiver is created from the session instance, otherwise it returns diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index d548d2049a5b..d74f3cae89fe 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -24,12 +24,8 @@ import java.io.IOException; import org.apache.beam.sdk.io.solace.RetryCallableManager; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SolaceMessageReceiver implements MessageReceiver { - private static final Logger LOG = LoggerFactory.getLogger(SolaceMessageReceiver.class); - public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; private final FlowReceiver flowReceiver; private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); @@ -52,19 +48,14 @@ private void startFlowReceiver() { ImmutableSet.of(JCSMPException.class)); } - @Override - public boolean isClosed() { - return flowReceiver == null || flowReceiver.isClosed(); - } - @Override public BytesXMLMessage receive() throws IOException { try { return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); } catch (StaleSessionException e) { - LOG.warn("SolaceIO: Caught StaleSessionException, restarting the FlowReceiver."); startFlowReceiver(); - throw new IOException(e); + throw new IOException( + "SolaceIO: Caught StaleSessionException, restarting the FlowReceiver.", e); } catch (JCSMPException e) { throw new IOException(e); } @@ -72,8 +63,6 @@ public BytesXMLMessage receive() throws IOException { @Override public void close() { - if (!isClosed()) { - this.flowReceiver.close(); - } + flowReceiver.close(); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 77f6eed8f62c..a913fd6133ea 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -18,17 +18,16 @@ package org.apache.beam.sdk.io.solace.read; import com.solacesystems.jcsmp.BytesXMLMessage; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Queue; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Checkpoint for an unbounded Solace source. Consists of the Solace messages waiting to be @@ -38,10 +37,8 @@ @Internal @VisibleForTesting public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private transient AtomicBoolean activeReader; - // BytesXMLMessage is not serializable so if a job restarts from the checkpoint, we cannot retry - // these messages here. We relay on Solace's retry mechanism. - private transient ArrayDeque ackQueue; + private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); + private transient Queue safeToAck; @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction private SolaceCheckpointMark() {} @@ -49,25 +46,24 @@ private SolaceCheckpointMark() {} /** * Creates a new {@link SolaceCheckpointMark}. * - * @param activeReader {@link AtomicBoolean} indicating if the related reader is active. The - * reader creating the messages has to be active to acknowledge the messages. - * @param ackQueue {@link List} of {@link BytesXMLMessage} to be acknowledged. + * @param safeToAck - a queue of {@link BytesXMLMessage} to be acknowledged. */ - SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { - this.activeReader = activeReader; - this.ackQueue = new ArrayDeque<>(ackQueue); + SolaceCheckpointMark(Queue safeToAck) { + this.safeToAck = safeToAck; } @Override public void finalizeCheckpoint() { - if (activeReader == null || !activeReader.get() || ackQueue == null) { - return; - } - - while (!ackQueue.isEmpty()) { - BytesXMLMessage msg = ackQueue.poll(); - if (msg != null) { + BytesXMLMessage msg; + while ((msg = safeToAck.poll()) != null) { + try { msg.ackMessage(); + } catch (IllegalStateException e) { + LOG.error( + "SolaceIO.Read: cannot acknowledge the message with applicationMessageId={}, ackMessageId={}. It will not be retried.", + msg.getApplicationMessageId(), + msg.getAckMessageId(), + e); } } } @@ -84,15 +80,11 @@ public boolean equals(@Nullable Object o) { return false; } SolaceCheckpointMark that = (SolaceCheckpointMark) o; - // Needed to convert to ArrayList because ArrayDeque.equals checks only for reference, not - // content. - ArrayList ackList = new ArrayList<>(ackQueue); - ArrayList thatAckList = new ArrayList<>(that.ackQueue); - return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); + return Objects.equals(safeToAck, that.safeToAck); } @Override public int hashCode() { - return Objects.hash(activeReader, ackQueue); + return Objects.hash(safeToAck); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index a421970370da..dc84e0a07017 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -22,17 +22,26 @@ import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; import java.util.NoSuchElementException; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Queue; +import java.util.UUID; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.io.solace.broker.SempClient; import org.apache.beam.sdk.io.solace.broker.SessionService; +import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalNotification; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; @@ -46,48 +55,92 @@ class UnboundedSolaceReader extends UnboundedReader { private final UnboundedSolaceSource currentSource; private final WatermarkPolicy watermarkPolicy; private final SempClient sempClient; + private final UUID readerUuid; + private final SessionServiceFactory sessionServiceFactory; private @Nullable BytesXMLMessage solaceOriginalRecord; private @Nullable T solaceMappedRecord; - private @Nullable SessionService sessionService; - AtomicBoolean active = new AtomicBoolean(true); /** - * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent - * queue, should only be accessed by the reader thread A given {@link UnboundedReader} object will - * only be accessed by a single thread at once. + * Queue to place advanced messages before {@link #getCheckpointMark()} is called. CAUTION: + * Accessed by both reader and checkpointing threads. */ - private final java.util.Queue elementsToCheckpoint = new ArrayDeque<>(); + private final Queue safeToAckMessages = new ConcurrentLinkedQueue<>(); + + /** + * Queue for messages that were ingested in the {@link #advance()} method, but not sent yet to a + * {@link SolaceCheckpointMark}. + */ + private final Queue receivedMessages = new ArrayDeque<>(); + + private static final Cache sessionServiceCache; + private static final ScheduledExecutorService cleanUpThread = Executors.newScheduledThreadPool(1); + + static { + Duration cacheExpirationTimeout = Duration.ofMinutes(1); + sessionServiceCache = + CacheBuilder.newBuilder() + .expireAfterAccess(cacheExpirationTimeout) + .removalListener( + (RemovalNotification notification) -> { + LOG.info( + "SolaceIO.Read: Closing session for the reader with uuid {} as it has been idle for over {}.", + notification.getKey(), + cacheExpirationTimeout); + SessionService sessionService = notification.getValue(); + if (sessionService != null) { + sessionService.close(); + } + }) + .build(); + + startCleanUpThread(); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private static void startCleanUpThread() { + cleanUpThread.scheduleAtFixedRate(sessionServiceCache::cleanUp, 1, 1, TimeUnit.MINUTES); + } public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { this.currentSource = currentSource; this.watermarkPolicy = WatermarkPolicy.create( currentSource.getTimestampFn(), currentSource.getWatermarkIdleDurationThreshold()); - this.sessionService = currentSource.getSessionServiceFactory().create(); + this.sessionServiceFactory = currentSource.getSessionServiceFactory(); this.sempClient = currentSource.getSempClientFactory().create(); + this.readerUuid = UUID.randomUUID(); + } + + private SessionService getSessionService() { + try { + return sessionServiceCache.get( + readerUuid, + () -> { + LOG.info("SolaceIO.Read: creating a new session for reader with uuid {}.", readerUuid); + SessionService sessionService = sessionServiceFactory.create(); + sessionService.connect(); + sessionService.getReceiver().start(); + return sessionService; + }); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } } @Override public boolean start() { - populateSession(); - checkNotNull(sessionService).getReceiver().start(); + // Create and initialize SessionService with Receiver + getSessionService(); return advance(); } - public void populateSession() { - if (sessionService == null) { - sessionService = getCurrentSource().getSessionServiceFactory().create(); - } - if (sessionService.isClosed()) { - checkNotNull(sessionService).connect(); - } - } - @Override public boolean advance() { + finalizeReadyMessages(); + BytesXMLMessage receivedXmlMessage; try { - receivedXmlMessage = checkNotNull(sessionService).getReceiver().receive(); + receivedXmlMessage = getSessionService().getReceiver().receive(); } catch (IOException e) { LOG.warn("SolaceIO.Read: Exception when pulling messages from the broker.", e); return false; @@ -96,23 +149,40 @@ public boolean advance() { if (receivedXmlMessage == null) { return false; } - elementsToCheckpoint.add(receivedXmlMessage); solaceOriginalRecord = receivedXmlMessage; solaceMappedRecord = getCurrentSource().getParseFn().apply(receivedXmlMessage); - watermarkPolicy.update(solaceMappedRecord); + receivedMessages.add(receivedXmlMessage); + return true; } @Override public void close() { - active.set(false); - checkNotNull(sessionService).close(); + finalizeReadyMessages(); + sessionServiceCache.invalidate(readerUuid); + } + + public void finalizeReadyMessages() { + BytesXMLMessage msg; + while ((msg = safeToAckMessages.poll()) != null) { + try { + msg.ackMessage(); + } catch (IllegalStateException e) { + LOG.error( + "SolaceIO.Read: failed to acknowledge the message with applicationMessageId={}, ackMessageId={}. Returning the message to queue to retry.", + msg.getApplicationMessageId(), + msg.getAckMessageId(), + e); + safeToAckMessages.add(msg); // In case the error was transient, might succeed later + break; // Commit is only best effort + } + } } @Override public Instant getWatermark() { // should be only used by a test receiver - if (checkNotNull(sessionService).getReceiver().isEOF()) { + if (getSessionService().getReceiver().isEOF()) { return BoundedWindow.TIMESTAMP_MAX_VALUE; } return watermarkPolicy.getWatermark(); @@ -120,14 +190,9 @@ public Instant getWatermark() { @Override public UnboundedSource.CheckpointMark getCheckpointMark() { - List ackQueue = new ArrayList<>(); - while (!elementsToCheckpoint.isEmpty()) { - BytesXMLMessage msg = elementsToCheckpoint.poll(); - if (msg != null) { - ackQueue.add(msg); - } - } - return new SolaceCheckpointMark(active, ackQueue); + safeToAckMessages.addAll(receivedMessages); + receivedMessages.clear(); + return new SolaceCheckpointMark(safeToAckMessages); } @Override diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockEmptySessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockEmptySessionService.java index 38b4953a5984..7631d32f63cc 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockEmptySessionService.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockEmptySessionService.java @@ -40,11 +40,6 @@ public void close() { throw new UnsupportedOperationException(exceptionMessage); } - @Override - public boolean isClosed() { - throw new UnsupportedOperationException(exceptionMessage); - } - @Override public MessageReceiver getReceiver() { throw new UnsupportedOperationException(exceptionMessage); diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java index bd52dee7ea86..6d28bcefc84c 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java @@ -77,11 +77,6 @@ public abstract Builder mockProducerFn( @Override public void close() {} - @Override - public boolean isClosed() { - return false; - } - @Override public MessageReceiver getReceiver() { if (messageReceiver == null) { @@ -131,11 +126,6 @@ public MockReceiver( @Override public void start() {} - @Override - public boolean isClosed() { - return false; - } - @Override public BytesXMLMessage receive() throws IOException { return getRecordFn.apply(counter.getAndIncrement()); diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOReadTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOReadTest.java index c718c55e1b48..a1f80932eddf 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOReadTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOReadTest.java @@ -447,25 +447,29 @@ public void testCheckpointMarkAndFinalizeSeparately() throws Exception { // start the reader and move to the first record assertTrue(reader.start()); - // consume 3 messages (NB: start already consumed the first message) + // consume 3 messages (NB: #start() already consumed the first message) for (int i = 0; i < 3; i++) { assertTrue(String.format("Failed at %d-th message", i), reader.advance()); } - // create checkpoint but don't finalize yet + // #advance() was called, but the messages were not ready to be acknowledged. + assertEquals(0, countAckMessages.get()); + + // mark all consumed messages as ready to be acknowledged CheckpointMark checkpointMark = reader.getCheckpointMark(); - // consume 2 more messages - reader.advance(); + // consume 1 more message. This will call #ackMsg() on messages that were ready to be acked. reader.advance(); + assertEquals(4, countAckMessages.get()); - // check if messages are still not acknowledged - assertEquals(0, countAckMessages.get()); + // consume 1 more message. No change in the acknowledged messages. + reader.advance(); + assertEquals(4, countAckMessages.get()); // acknowledge from the first checkpoint checkpointMark.finalizeCheckpoint(); - - // only messages from the first checkpoint are acknowledged + // No change in the acknowledged messages, because they were acknowledged in the #advance() + // method. assertEquals(4, countAckMessages.get()); } From a27282363aa2b97dda96750f6d1cbf55ebd752ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rados=C5=82aw=20Stankiewicz?= Date: Wed, 11 Dec 2024 20:35:42 +0100 Subject: [PATCH 17/33] Update confluent version to fix CVE-2024-26308 CVE-2024-25710 (#32674) * bump confluent version Kafka Schema Registry Client has been reported with following vuln CVE-2024-26308 CVE-2024-25710 due to vulnerable dependencies. * try slighly older version due to unmet dependencies to ThrottlingQuotaExceededException * try slighly older version due to unmet dependencies to ThrottlingQuotaExceededException * comment on version --- sdks/java/io/kafka/build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index c2f056b0b7cb..04563c478d6d 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -31,7 +31,8 @@ enableJavaPerformanceTesting() description = "Apache Beam :: SDKs :: Java :: IO :: Kafka" ext { summary = "Library to read Kafka topics." - confluentVersion = "7.6.0" + // newer versions e.g. 7.6.* require dropping support for older kafka versions. + confluentVersion = "7.5.5" } def kafkaVersions = [ From 3dcd10424f66e20ec746eb9f7ff3122cf716c53c Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 11 Dec 2024 11:52:45 -0800 Subject: [PATCH 18/33] Update reference. --- sdks/python/apache_beam/metrics/cells.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py index e044693a6e9a..8b4b7e1271c2 100644 --- a/sdks/python/apache_beam/metrics/cells.py +++ b/sdks/python/apache_beam/metrics/cells.py @@ -318,7 +318,7 @@ def to_runner_api_monitoring_info_impl(self, name, transform_id): class BoundedTrieCell(AbstractMetricCell): """For internal use only; no backwards-compatibility guarantees. - Tracks the current value for a StringSet metric. + Tracks the current value for a BoundedTrie metric. Each cell tracks the state of a metric independently per context per bundle. Therefore, each metric has a different cell in each bundle, that is later From bb0c910b195bc44bcef0e268a8b08cc25ab44ef8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Dec 2024 10:27:43 -0500 Subject: [PATCH 19/33] Bump cloud.google.com/go/profiler from 0.4.1 to 0.4.2 in /sdks (#33351) Bumps [cloud.google.com/go/profiler](https://github.com/googleapis/google-cloud-go) from 0.4.1 to 0.4.2. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/ai/v0.4.1...apps/v0.4.2) --- updated-dependencies: - dependency-name: cloud.google.com/go/profiler dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5406c2b70cbc..30433676f6f4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -26,7 +26,7 @@ require ( cloud.google.com/go/bigquery v1.64.0 cloud.google.com/go/bigtable v1.33.0 cloud.google.com/go/datastore v1.20.0 - cloud.google.com/go/profiler v0.4.1 + cloud.google.com/go/profiler v0.4.2 cloud.google.com/go/pubsub v1.45.3 cloud.google.com/go/spanner v1.73.0 cloud.google.com/go/storage v1.47.0 @@ -159,7 +159,7 @@ require ( github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/flatbuffers v23.5.26+incompatible // indirect - github.com/google/pprof v0.0.0-20240528025155-186aa0362fba // indirect + github.com/google/pprof v0.0.0-20240727154555-813a5fbdbec8 // indirect github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.8 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.4 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index bb96c54af087..6394ce515c4d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -441,8 +441,8 @@ cloud.google.com/go/privatecatalog v0.5.0/go.mod h1:XgosMUvvPyxDjAVNDYxJ7wBW8//h cloud.google.com/go/privatecatalog v0.6.0/go.mod h1:i/fbkZR0hLN29eEWiiwue8Pb+GforiEIBnV9yrRUOKI= cloud.google.com/go/privatecatalog v0.7.0/go.mod h1:2s5ssIFO69F5csTXcwBP7NPFTZvps26xGzvQ2PQaBYg= cloud.google.com/go/privatecatalog v0.8.0/go.mod h1:nQ6pfaegeDAq/Q5lrfCQzQLhubPiZhSaNhIgfJlnIXs= -cloud.google.com/go/profiler v0.4.1 h1:Q7+lOvikTGMJ/IAWocpYYGit4SIIoILmVZfEEWTORSY= -cloud.google.com/go/profiler v0.4.1/go.mod h1:LBrtEX6nbvhv1w/e5CPZmX9ajGG9BGLtGbv56Tg4SHs= +cloud.google.com/go/profiler v0.4.2 h1:KojCmZ+bEPIQrd7bo2UFvZ2xUPLHl55KzHl7iaR4V2I= +cloud.google.com/go/profiler v0.4.2/go.mod h1:7GcWzs9deJHHdJ5J9V1DzKQ9JoIoTGhezwlLbwkOoCs= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -948,8 +948,8 @@ github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20240528025155-186aa0362fba h1:ql1qNgCyOB7iAEk8JTNM+zJrgIbnyCKX/wdlyPufP5g= -github.com/google/pprof v0.0.0-20240528025155-186aa0362fba/go.mod h1:K1liHPHnj73Fdn/EKuT8nrFqBihUSKXoLYU0BuatOYo= +github.com/google/pprof v0.0.0-20240727154555-813a5fbdbec8 h1:FKHo8hFI3A+7w0aUQuYXQ+6EN5stWmeY/AZqtM8xk9k= +github.com/google/pprof v0.0.0-20240727154555-813a5fbdbec8/go.mod h1:K1liHPHnj73Fdn/EKuT8nrFqBihUSKXoLYU0BuatOYo= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg= github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4= From 224900c42a6723ec4d08b1acae6d3e16f41ddb0a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Dec 2024 10:28:14 -0500 Subject: [PATCH 20/33] Bump cloud.google.com/go/storage from 1.47.0 to 1.48.0 in /sdks (#33327) Bumps [cloud.google.com/go/storage](https://github.com/googleapis/google-cloud-go) from 1.47.0 to 1.48.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.47.0...spanner/v1.48.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/storage dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 30433676f6f4..9eda64804cb6 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -29,7 +29,7 @@ require ( cloud.google.com/go/profiler v0.4.2 cloud.google.com/go/pubsub v1.45.3 cloud.google.com/go/spanner v1.73.0 - cloud.google.com/go/storage v1.47.0 + cloud.google.com/go/storage v1.48.0 github.com/aws/aws-sdk-go-v2 v1.32.6 github.com/aws/aws-sdk-go-v2/config v1.28.6 github.com/aws/aws-sdk-go-v2/credentials v1.17.47 @@ -60,7 +60,7 @@ require ( golang.org/x/text v0.21.0 google.golang.org/api v0.210.0 google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 - google.golang.org/grpc v1.67.1 + google.golang.org/grpc v1.67.2 google.golang.org/protobuf v1.35.2 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 diff --git a/sdks/go.sum b/sdks/go.sum index 6394ce515c4d..eae0e0007706 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -561,8 +561,8 @@ cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeL cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= cloud.google.com/go/storage v1.29.0/go.mod h1:4puEjyTKnku6gfKoTfNOU/W+a9JyuVNxjpS5GBrB8h4= -cloud.google.com/go/storage v1.47.0 h1:ajqgt30fnOMmLfWfu1PWcb+V9Dxz6n+9WKjdNg5R4HM= -cloud.google.com/go/storage v1.47.0/go.mod h1:Ks0vP374w0PW6jOUameJbapbQKXqkjGd/OJRp2fb9IQ= +cloud.google.com/go/storage v1.48.0 h1:FhBDHACbVtdPx7S/AbcKujPWiHvfO6F8OXGgCEbB2+o= +cloud.google.com/go/storage v1.48.0/go.mod h1:aFoDYNMAjv67lp+xcuZqjUKv/ctmplzQ3wJgodA7b+M= cloud.google.com/go/storagetransfer v1.5.0/go.mod h1:dxNzUopWy7RQevYFHewchb29POFv3/AaBgnhqzqiK0w= cloud.google.com/go/storagetransfer v1.6.0/go.mod h1:y77xm4CQV/ZhFZH75PLEXY0ROiS7Gh6pSKrM8dJyg6I= cloud.google.com/go/storagetransfer v1.7.0/go.mod h1:8Giuj1QNb1kfLAiWM1bN6dHzfdlDAVC9rv9abHot2W4= @@ -1895,8 +1895,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= -google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/grpc v1.67.2 h1:Lq11HW1nr5m4OYV+ZVy2BjOK78/zqnTx24vyDBP1JcQ= +google.golang.org/grpc v1.67.2/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/grpc/stats/opentelemetry v0.0.0-20240907200651-3ffb98b2c93a h1:UIpYSuWdWHSzjwcAFRLjKcPXFZVVLXGEM23W+NWqipw= google.golang.org/grpc/stats/opentelemetry v0.0.0-20240907200651-3ffb98b2c93a/go.mod h1:9i1T9n4ZinTUZGgzENMi8MDDgbGC5mqTS75JAv6xN3A= From 272feb8ff69a1d6174d04693347ca9bd2ac1980b Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Thu, 12 Dec 2024 10:30:38 -0500 Subject: [PATCH 21/33] Fix custom coder not being used in Reshuffle (global window) (#33339) * Fix typehint in ReshufflePerKey on global window setting. * Only update the type hint on global window setting. Need more work in non-global windows. * Apply yapf * Fix some failed tests. * Revert change to setup.py --- sdks/python/apache_beam/transforms/util.py | 13 ++++++- .../apache_beam/transforms/util_test.py | 39 +++++++++++++++++++ 2 files changed, 50 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index a03652de2496..43d4a6c20e94 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -33,6 +33,7 @@ from typing import Callable from typing import Iterable from typing import List +from typing import Optional from typing import Tuple from typing import TypeVar from typing import Union @@ -78,6 +79,7 @@ from apache_beam.utils import windowed_value from apache_beam.utils.annotations import deprecated from apache_beam.utils.sharded_key import ShardedKey +from apache_beam.utils.timestamp import Timestamp if TYPE_CHECKING: from apache_beam.runners.pipeline_context import PipelineContext @@ -953,6 +955,10 @@ def restore_timestamps(element): window.GlobalWindows.windowed_value((key, value), timestamp) for (value, timestamp) in values ] + + ungrouped = pcoll | Map(reify_timestamps).with_input_types( + Tuple[K, V]).with_output_types( + Tuple[K, Tuple[V, Optional[Timestamp]]]) else: # typing: All conditional function variants must have identical signatures @@ -966,7 +972,9 @@ def restore_timestamps(element): key, windowed_values = element return [wv.with_value((key, wv.value)) for wv in windowed_values] - ungrouped = pcoll | Map(reify_timestamps).with_output_types(Any) + # TODO(https://github.com/apache/beam/issues/33356): Support reshuffling + # unpicklable objects with a non-global window setting. + ungrouped = pcoll | Map(reify_timestamps).with_output_types(Any) # TODO(https://github.com/apache/beam/issues/19785) Using global window as # one of the standard window. This is to mitigate the Dataflow Java Runner @@ -1018,7 +1026,8 @@ def expand(self, pcoll): pcoll | 'AddRandomKeys' >> Map(lambda t: (random.randrange(0, self.num_buckets), t) ).with_input_types(T).with_output_types(Tuple[int, T]) - | ReshufflePerKey() + | ReshufflePerKey().with_input_types(Tuple[int, T]).with_output_types( + Tuple[int, T]) | 'RemoveRandomKeys' >> Map(lambda t: t[1]).with_input_types( Tuple[int, T]).with_output_types(T)) diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index d86509c7dde3..7f166f78ef0a 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -1010,6 +1010,45 @@ def format_with_timestamp(element, timestamp=beam.DoFn.TimestampParam): equal_to(expected_data), label="formatted_after_reshuffle") + def test_reshuffle_unpicklable_in_global_window(self): + global _Unpicklable + + class _Unpicklable(object): + def __init__(self, value): + self.value = value + + def __getstate__(self): + raise NotImplementedError() + + def __setstate__(self, state): + raise NotImplementedError() + + class _UnpicklableCoder(beam.coders.Coder): + def encode(self, value): + return str(value.value).encode() + + def decode(self, encoded): + return _Unpicklable(int(encoded.decode())) + + def to_type_hint(self): + return _Unpicklable + + def is_deterministic(self): + return True + + beam.coders.registry.register_coder(_Unpicklable, _UnpicklableCoder) + + with TestPipeline() as pipeline: + data = [_Unpicklable(i) for i in range(5)] + expected_data = [0, 10, 20, 30, 40] + result = ( + pipeline + | beam.Create(data) + | beam.WindowInto(GlobalWindows()) + | beam.Reshuffle() + | beam.Map(lambda u: u.value * 10)) + assert_that(result, equal_to(expected_data)) + class WithKeysTest(unittest.TestCase): def setUp(self): From 7b883373d4a9dc69e10949f0c1b1c7479df62d48 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Dec 2024 09:36:54 -0800 Subject: [PATCH 22/33] Bump golang.org/x/crypto from 0.30.0 to 0.31.0 in /sdks (#33366) Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.30.0 to 0.31.0. - [Commits](https://github.com/golang/crypto/compare/v0.30.0...v0.31.0) --- updated-dependencies: - dependency-name: golang.org/x/crypto dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9eda64804cb6..cd993672663b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -190,7 +190,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.30.0 // indirect + golang.org/x/crypto v0.31.0 // indirect golang.org/x/mod v0.20.0 // indirect golang.org/x/tools v0.24.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect diff --git a/sdks/go.sum b/sdks/go.sum index eae0e0007706..a95dcac4913c 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1266,8 +1266,8 @@ golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= -golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= +golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= From fd17dcea9cae8a444fcd1fc9708dffdd2e33cda0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Dec 2024 13:01:10 -0500 Subject: [PATCH 23/33] Update numpy requirement from <2.2.0,>=1.14.3 to >=1.14.3,<2.3.0 in /sdks/python (#33325) * Update numpy requirement in /sdks/python Updates the requirements on [numpy](https://github.com/numpy/numpy) to permit the latest version. - [Release notes](https://github.com/numpy/numpy/releases) - [Changelog](https://github.com/numpy/numpy/blob/main/doc/RELEASE_WALKTHROUGH.rst) - [Commits](https://github.com/numpy/numpy/compare/v1.14.3...v2.2.0) --- updated-dependencies: - dependency-name: numpy dependency-type: direct:production ... Signed-off-by: dependabot[bot] * increment in setup.py --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Jack McCluskey --- sdks/python/pyproject.toml | 2 +- sdks/python/setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/pyproject.toml b/sdks/python/pyproject.toml index 4eb827297019..8000c24f28aa 100644 --- a/sdks/python/pyproject.toml +++ b/sdks/python/pyproject.toml @@ -26,7 +26,7 @@ requires = [ # Avoid https://github.com/pypa/virtualenv/issues/2006 "distlib==0.3.7", # Numpy headers - "numpy>=1.14.3,<2.2.0", # Update setup.py as well. + "numpy>=1.14.3,<2.3.0", # Update setup.py as well. # having cython here will create wheels that are platform dependent. "cython>=3.0,<4", ## deps for generating external transform wrappers: diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 53c7a532e706..da9e0b2e7477 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -361,7 +361,7 @@ def get_portability_package_data(): 'jsonpickle>=3.0.0,<4.0.0', # numpy can have breaking changes in minor versions. # Use a strict upper bound. - 'numpy>=1.14.3,<2.2.0', # Update pyproject.toml as well. + 'numpy>=1.14.3,<2.3.0', # Update pyproject.toml as well. 'objsize>=0.6.1,<0.8.0', 'packaging>=22.0', 'pymongo>=3.8.0,<5.0.0', From 2efd124429b4810dcc1d7f622f48c0fed7fc3c48 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 12 Dec 2024 14:09:40 -0500 Subject: [PATCH 24/33] Switch to use unshaded hive-exec for io expansion service (#33349) * Switch to use unshaded hive-exec for io expansion service * This enables the shadow jar pick up dependencies of newer versions * cleanup leftovers --- .../IO_Iceberg_Integration_Tests.json | 2 +- ...eam_PostCommit_Python_Xlang_IO_Direct.json | 2 +- sdks/java/io/expansion-service/build.gradle | 2 +- sdks/java/io/iceberg/hive/build.gradle | 33 ++++++++-- sdks/java/io/iceberg/hive/exec/build.gradle | 65 ------------------- settings.gradle.kts | 2 - 6 files changed, 31 insertions(+), 75 deletions(-) delete mode 100644 sdks/java/io/iceberg/hive/exec/build.gradle diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 3f63c0c9975f..bbdc3a3910ef 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 2 + "modification": 3 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json index b26833333238..e3d6056a5de9 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 2 + "modification": 1 } diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index 421719b8f986..a27a66b1f3dc 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -60,7 +60,7 @@ dependencies { runtimeOnly library.java.bigdataoss_gcs_connector // Needed for HiveCatalog runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") - runtimeOnly project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") + runtimeOnly project(path: ":sdks:java:io:iceberg:hive") runtimeOnly library.java.kafka_clients runtimeOnly library.java.slf4j_jdk14 diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle index bfa6c75251c4..2d0d2bcc5cde 100644 --- a/sdks/java/io/iceberg/hive/build.gradle +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -21,19 +21,39 @@ plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.iceberg.hive', exportJavadoc: false, - shadowClosure: {}, + publish: false, // it's an intermediate jar for io-expansion-service ) description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive" ext.summary = "Runtime dependencies needed for Hive catalog integration." def hive_version = "3.1.3" +def hbase_version = "2.6.1-hadoop3" +def hadoop_version = "3.4.1" def iceberg_version = "1.4.2" +def avatica_version = "1.25.0" dependencies { // dependencies needed to run with iceberg's hive catalog - runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") - runtimeOnly project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") + // these dependencies are going to be included in io-expansion-service + implementation ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") + permitUnusedDeclared ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") + // analyzeClassesDependencies fails with "Cannot accept visitor on URL", likely the plugin does not recognize "core" classifier + // use "core" classifier to depend on un-shaded jar + runtimeOnly ("org.apache.hive:hive-exec:$hive_version:core") { + // old hadoop-yarn-server-resourcemanager contains critical log4j vulneribility + exclude group: "org.apache.hadoop", module: "hadoop-yarn-server-resourcemanager" + // old hadoop-yarn-server-resourcemanager contains critical log4j and hadoop vulneribility + exclude group: "org.apache.hbase", module: "hbase-client" + // old calcite leaks old protobuf-java + exclude group: "org.apache.calcite.avatica", module: "avatica" + } + runtimeOnly ("org.apache.hadoop:hadoop-yarn-server-resourcemanager:$hadoop_version") + runtimeOnly ("org.apache.hbase:hbase-client:$hbase_version") + runtimeOnly ("org.apache.calcite.avatica:avatica-core:$avatica_version") + implementation ("org.apache.hive:hive-metastore:$hive_version") + runtimeOnly ("org.apache.iceberg:iceberg-parquet:$iceberg_version") + permitUnusedDeclared ("org.apache.hive:hive-metastore:$hive_version") // ----- below dependencies are for testing and will not appear in the shaded jar ----- // Beam IcebergIO dependencies @@ -52,8 +72,6 @@ dependencies { testImplementation library.java.junit // needed to set up test Hive Metastore and run tests - testImplementation ("org.apache.iceberg:iceberg-hive-metastore:$iceberg_version") - testImplementation project(path: ":sdks:java:io:iceberg:hive:exec", configuration: "shadow") testRuntimeOnly ("org.apache.hive.hcatalog:hive-hcatalog-core:$hive_version") { exclude group: "org.apache.hive", module: "hive-exec" exclude group: "org.apache.parquet", module: "parquet-hadoop-bundle" @@ -62,6 +80,11 @@ dependencies { testImplementation "org.apache.parquet:parquet-column:1.12.0" } +configurations.all { + // the fatjar "parquet-hadoop-bundle" conflicts with "parquet-hadoop" used by org.apache.iceberg:iceberg-parquet + exclude group: "org.apache.parquet", module: "parquet-hadoop-bundle" +} + task integrationTest(type: Test) { group = "Verification" def gcpTempLocation = project.findProperty('gcpTempLocation') ?: 'gs://temp-storage-for-end-to-end-tests/iceberg-hive-it' diff --git a/sdks/java/io/iceberg/hive/exec/build.gradle b/sdks/java/io/iceberg/hive/exec/build.gradle deleted file mode 100644 index f266ab2ef4db..000000000000 --- a/sdks/java/io/iceberg/hive/exec/build.gradle +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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. - */ -plugins { - id 'org.apache.beam.module' - id 'java' - id 'com.github.johnrengelman.shadow' -} - -dependencies { - implementation("org.apache.hive:hive-exec:3.1.3") - permitUnusedDeclared("org.apache.hive:hive-exec:3.1.3") -} - -configurations { - shadow -} - -artifacts { - shadow(archives(shadowJar) { - builtBy shadowJar - }) -} - -shadowJar { - zip64 true - - def problematicPackages = [ - 'com.google.protobuf', - 'com.google.common', - 'shaded.parquet', - 'org.apache.parquet', - 'org.joda' - ] - - problematicPackages.forEach { - relocate it, getJavaRelocatedPath("iceberg.hive.${it}") - } - - version "3.1.3" - mergeServiceFiles() - - exclude 'LICENSE' - exclude( - 'org/xml/**', - 'javax/**', - 'com/sun/**' - ) -} -description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive :: Exec" -ext.summary = "A copy of the hive-exec dependency with some popular libraries relocated." diff --git a/settings.gradle.kts b/settings.gradle.kts index d90bb3fb5b82..a8bee45a05ac 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -357,5 +357,3 @@ include("sdks:java:extensions:combiners") findProject(":sdks:java:extensions:combiners")?.name = "combiners" include("sdks:java:io:iceberg:hive") findProject(":sdks:java:io:iceberg:hive")?.name = "hive" -include("sdks:java:io:iceberg:hive:exec") -findProject(":sdks:java:io:iceberg:hive:exec")?.name = "exec" From e9c0d35ff3390a3ff4cca3562508597dc3a69a65 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Dec 2024 14:21:13 -0500 Subject: [PATCH 25/33] Bump cloud.google.com/go/bigquery from 1.64.0 to 1.65.0 in /sdks (#33365) Bumps [cloud.google.com/go/bigquery](https://github.com/googleapis/google-cloud-go) from 1.64.0 to 1.65.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.64.0...spanner/v1.65.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigquery dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index cd993672663b..e6a182035f5e 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -23,7 +23,7 @@ module github.com/apache/beam/sdks/v2 go 1.21.0 require ( - cloud.google.com/go/bigquery v1.64.0 + cloud.google.com/go/bigquery v1.65.0 cloud.google.com/go/bigtable v1.33.0 cloud.google.com/go/datastore v1.20.0 cloud.google.com/go/profiler v0.4.2 diff --git a/sdks/go.sum b/sdks/go.sum index a95dcac4913c..34e34ef2f8cd 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -133,8 +133,8 @@ cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/Zur cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= -cloud.google.com/go/bigquery v1.64.0 h1:vSSZisNyhr2ioJE1OuYBQrnrpB7pIhRQm4jfjc7E/js= -cloud.google.com/go/bigquery v1.64.0/go.mod h1:gy8Ooz6HF7QmA+TRtX8tZmXBKH5mCFBwUApGAb3zI7Y= +cloud.google.com/go/bigquery v1.65.0 h1:ZZ1EOJMHTYf6R9lhxIXZJic1qBD4/x9loBIS+82moUs= +cloud.google.com/go/bigquery v1.65.0/go.mod h1:9WXejQ9s5YkTW4ryDYzKXBooL78u5+akWGXgJqQkY6A= cloud.google.com/go/bigtable v1.33.0 h1:2BDaWLRAwXO14DJL/u8crbV2oUbMZkIa2eGq8Yao1bk= cloud.google.com/go/bigtable v1.33.0/go.mod h1:HtpnH4g25VT1pejHRtInlFPnN5sjTxbQlsYBjh9t5l0= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= From de5e8ebcee290a00036c868a1f444018bd445506 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 12 Dec 2024 13:04:26 -0800 Subject: [PATCH 26/33] [#32211] Support OnWindowExpiration in Prism. (#33337) --- CHANGES.md | 2 + runners/prism/java/build.gradle | 18 +- .../prism/internal/engine/elementmanager.go | 224 +++++++++++++++--- .../internal/engine/elementmanager_test.go | 160 +++++++++++++ .../beam/runners/prism/internal/execute.go | 4 + .../runners/prism/internal/jobservices/job.go | 1 + .../prism/internal/jobservices/management.go | 2 - .../beam/runners/prism/internal/preprocess.go | 3 + .../pkg/beam/runners/prism/internal/stage.go | 24 +- .../runners/prism/internal/worker/bundle.go | 4 +- 10 files changed, 388 insertions(+), 54 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index dbadd588ae3f..ea29b6aadc7d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -69,6 +69,8 @@ ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Support OnWindowExpiration in Prism ([#32211](https://github.com/apache/beam/issues/32211)). + * This enables initial Java GroupIntoBatches support. ## Breaking Changes diff --git a/runners/prism/java/build.gradle b/runners/prism/java/build.gradle index 82eb62b9e207..ce71151099bd 100644 --- a/runners/prism/java/build.gradle +++ b/runners/prism/java/build.gradle @@ -106,6 +106,12 @@ def sickbayTests = [ 'org.apache.beam.sdk.testing.TestStreamTest.testMultipleStreams', 'org.apache.beam.sdk.testing.TestStreamTest.testProcessingTimeTrigger', + // GroupIntoBatchesTest tests that fail: + // Teststream has bad KV encodings due to using an outer context. + 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testInStreamingMode', + // ShardedKey not yet implemented. + 'org.apache.beam.sdk.transforms.GroupIntoBatchesTest.testWithShardedKeyInGlobalWindow', + // Coding error somehow: short write: reached end of stream after reading 5 bytes; 98 bytes expected 'org.apache.beam.sdk.testing.TestStreamTest.testMultiStage', @@ -228,14 +234,16 @@ def createPrismValidatesRunnerTask = { name, environmentType -> excludeCategories 'org.apache.beam.sdk.testing.UsesSdkHarnessEnvironment' // Not yet implemented in Prism - // https://github.com/apache/beam/issues/32211 - excludeCategories 'org.apache.beam.sdk.testing.UsesOnWindowExpiration' // https://github.com/apache/beam/issues/32929 excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState' - // Not supported in Portable Java SDK yet. - // https://github.com/apache/beam/issues?q=is%3Aissue+is%3Aopen+MultimapState - excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState' + // Not supported in Portable Java SDK yet. + // https://github.com/apache/beam/issues?q=is%3Aissue+is%3Aopen+MultimapState + excludeCategories 'org.apache.beam.sdk.testing.UsesMultimapState' + + // Processing time with TestStream is unreliable without being able to control + // SDK side time portably. Ignore these tests. + excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime' } filter { for (String test : sickbayTests) { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 1739efdb742a..00e18c669afa 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -184,7 +184,8 @@ type Config struct { // // Watermarks are advanced based on consumed input, except if the stage produces residuals. type ElementManager struct { - config Config + config Config + nextBundID func() string // Generates unique bundleIDs. Set in the Bundles method. impulses set[string] // List of impulse stages. stages map[string]*stageState // The state for each stage. @@ -197,6 +198,7 @@ type ElementManager struct { refreshCond sync.Cond // refreshCond protects the following fields with it's lock, and unblocks bundle scheduling. inprogressBundles set[string] // Active bundleIDs changedStages set[string] // Stages that have changed and need their watermark refreshed. + injectedBundles []RunBundle // Represents ready to execute bundles prepared outside of the main loop, such as for onWindowExpiration, or for Triggers. livePending atomic.Int64 // An accessible live pending count. DEBUG USE ONLY pendingElements sync.WaitGroup // pendingElements counts all unprocessed elements in a job. Jobs with no pending elements terminate successfully. @@ -271,6 +273,16 @@ func (em *ElementManager) StageStateful(ID string) { em.stages[ID].stateful = true } +// StageOnWindowExpiration marks the given stage as stateful, which means elements are +// processed by key. +func (em *ElementManager) StageOnWindowExpiration(stageID string, timer StaticTimerID) { + ss := em.stages[stageID] + ss.onWindowExpiration = timer + ss.keysToExpireByWindow = map[typex.Window]set[string]{} + ss.inProgressExpiredWindows = map[typex.Window]int{} + ss.expiryWindowsByBundles = map[string]typex.Window{} +} + // StageProcessingTimeTimers indicates which timers are processingTime domain timers. func (em *ElementManager) StageProcessingTimeTimers(ID string, ptTimers map[string]bool) { em.stages[ID].processingTimeTimersFamilies = ptTimers @@ -338,6 +350,8 @@ func (rb RunBundle) LogValue() slog.Value { // The returned channel is closed when the context is canceled, or there are no pending elements // remaining. func (em *ElementManager) Bundles(ctx context.Context, upstreamCancelFn context.CancelCauseFunc, nextBundID func() string) <-chan RunBundle { + // Make it easier for injected bundles to get unique IDs. + em.nextBundID = nextBundID runStageCh := make(chan RunBundle) ctx, cancelFn := context.WithCancelCause(ctx) go func() { @@ -370,8 +384,9 @@ func (em *ElementManager) Bundles(ctx context.Context, upstreamCancelFn context. changedByProcessingTime := em.processTimeEvents.AdvanceTo(emNow) em.changedStages.merge(changedByProcessingTime) - // If there are no changed stages or ready processing time events available, we wait until there are. - for len(em.changedStages)+len(changedByProcessingTime) == 0 { + // If there are no changed stages, ready processing time events, + // or injected bundles available, we wait until there are. + for len(em.changedStages)+len(changedByProcessingTime)+len(em.injectedBundles) == 0 { // Check to see if we must exit select { case <-ctx.Done(): @@ -386,6 +401,19 @@ func (em *ElementManager) Bundles(ctx context.Context, upstreamCancelFn context. changedByProcessingTime = em.processTimeEvents.AdvanceTo(emNow) em.changedStages.merge(changedByProcessingTime) } + // Run any injected bundles first. + for len(em.injectedBundles) > 0 { + rb := em.injectedBundles[0] + em.injectedBundles = em.injectedBundles[1:] + em.refreshCond.L.Unlock() + + select { + case <-ctx.Done(): + return + case runStageCh <- rb: + } + em.refreshCond.L.Lock() + } // We know there is some work we can do that may advance the watermarks, // refresh them, and see which stages have advanced. @@ -628,6 +656,12 @@ type Block struct { Transform, Family string } +// StaticTimerID represents the static user identifiers for a timer, +// in particular, the ID of the Transform, and the family for the timer. +type StaticTimerID struct { + TransformID, TimerFamily string +} + // StateForBundle retreives relevant state for the given bundle, WRT the data in the bundle. // // TODO(lostluck): Consider unifiying with InputForBundle, to reduce lock contention. @@ -847,6 +881,19 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol } delete(stage.inprogressHoldsByBundle, rb.BundleID) + // Clean up OnWindowExpiration bundle accounting, so window state + // may be garbage collected. + if stage.expiryWindowsByBundles != nil { + win, ok := stage.expiryWindowsByBundles[rb.BundleID] + if ok { + stage.inProgressExpiredWindows[win] -= 1 + if stage.inProgressExpiredWindows[win] == 0 { + delete(stage.inProgressExpiredWindows, win) + } + delete(stage.expiryWindowsByBundles, rb.BundleID) + } + } + // If there are estimated output watermarks, set the estimated // output watermark for the stage. if len(residuals.MinOutputWatermarks) > 0 { @@ -1068,6 +1115,12 @@ type stageState struct { strat winStrat // Windowing Strategy for aggregation fireings. processingTimeTimersFamilies map[string]bool // Indicates which timer families use the processing time domain. + // onWindowExpiration management + onWindowExpiration StaticTimerID // The static ID of the OnWindowExpiration callback. + keysToExpireByWindow map[typex.Window]set[string] // Tracks all keys ever used with a window, so they may be expired. + inProgressExpiredWindows map[typex.Window]int // Tracks the number of bundles currently expiring these windows, so we don't prematurely garbage collect them. + expiryWindowsByBundles map[string]typex.Window // Tracks which bundle is handling which window, so the above map can be cleared. + mu sync.Mutex upstreamWatermarks sync.Map // watermark set from inputPCollection's parent. input mtime.Time // input watermark for the parallel input. @@ -1158,6 +1211,14 @@ func (ss *stageState) AddPending(newPending []element) int { timers: map[timerKey]timerTimes{}, } ss.pendingByKeys[string(e.keyBytes)] = dnt + if ss.keysToExpireByWindow != nil { + w, ok := ss.keysToExpireByWindow[e.window] + if !ok { + w = make(set[string]) + ss.keysToExpireByWindow[e.window] = w + } + w.insert(string(e.keyBytes)) + } } heap.Push(&dnt.elements, e) @@ -1555,48 +1616,143 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] { if minWatermarkHold < newOut { newOut = minWatermarkHold } - refreshes := set[string]{} + // If the newOut is smaller, then don't change downstream watermarks. + if newOut <= ss.output { + return nil + } + // If bigger, advance the output watermark - if newOut > ss.output { - ss.output = newOut - for _, outputCol := range ss.outputIDs { - consumers := em.consumers[outputCol] - - for _, sID := range consumers { - em.stages[sID].updateUpstreamWatermark(outputCol, ss.output) - refreshes.insert(sID) - } - // Inform side input consumers, but don't update the upstream watermark. - for _, sID := range em.sideConsumers[outputCol] { - refreshes.insert(sID.Global) - } - } - // Garbage collect state, timers and side inputs, for all windows - // that are before the new output watermark. - // They'll never be read in again. - for _, wins := range ss.sideInputs { - for win := range wins { - // TODO(#https://github.com/apache/beam/issues/31438): - // Adjust with AllowedLateness - // Clear out anything we've already used. - if win.MaxTimestamp() < newOut { - delete(wins, win) + preventDownstreamUpdate := ss.createOnWindowExpirationBundles(newOut, em) + + // Garbage collect state, timers and side inputs, for all windows + // that are before the new output watermark, if they aren't in progress + // of being expired. + // They'll never be read in again. + for _, wins := range ss.sideInputs { + for win := range wins { + // TODO(#https://github.com/apache/beam/issues/31438): + // Adjust with AllowedLateness + // Clear out anything we've already used. + if win.MaxTimestamp() < newOut { + // If the expiry is in progress, skip this window. + if ss.inProgressExpiredWindows[win] > 0 { + continue } + delete(wins, win) } } - for _, wins := range ss.state { - for win := range wins { - // TODO(#https://github.com/apache/beam/issues/31438): - // Adjust with AllowedLateness - if win.MaxTimestamp() < newOut { - delete(wins, win) + } + for _, wins := range ss.state { + for win := range wins { + // TODO(#https://github.com/apache/beam/issues/31438): + // Adjust with AllowedLateness + if win.MaxTimestamp() < newOut { + // If the expiry is in progress, skip collecting this window. + if ss.inProgressExpiredWindows[win] > 0 { + continue } + delete(wins, win) } } } + // If there are windows to expire, we don't update the output watermark yet. + if preventDownstreamUpdate { + return nil + } + + // Update this stage's output watermark, and then propagate that to downstream stages + refreshes := set[string]{} + ss.output = newOut + for _, outputCol := range ss.outputIDs { + consumers := em.consumers[outputCol] + + for _, sID := range consumers { + em.stages[sID].updateUpstreamWatermark(outputCol, ss.output) + refreshes.insert(sID) + } + // Inform side input consumers, but don't update the upstream watermark. + for _, sID := range em.sideConsumers[outputCol] { + refreshes.insert(sID.Global) + } + } return refreshes } +// createOnWindowExpirationBundles injects bundles when windows +// expire for all keys that were used in that window. Returns true if any +// bundles are created, which means that the window must not yet be garbage +// collected. +// +// Must be called within the stageState.mu's and the ElementManager.refreshCond +// critical sections. +func (ss *stageState) createOnWindowExpirationBundles(newOut mtime.Time, em *ElementManager) bool { + var preventDownstreamUpdate bool + for win, keys := range ss.keysToExpireByWindow { + // Check if the window has expired. + // TODO(#https://github.com/apache/beam/issues/31438): + // Adjust with AllowedLateness + if win.MaxTimestamp() >= newOut { + continue + } + // We can't advance the output watermark if there's garbage to collect. + preventDownstreamUpdate = true + // Hold off on garbage collecting data for these windows while these + // are in progress. + ss.inProgressExpiredWindows[win] += 1 + + // Produce bundle(s) for these keys and window, and inject them. + wm := win.MaxTimestamp() + rb := RunBundle{StageID: ss.ID, BundleID: "owe-" + em.nextBundID(), Watermark: wm} + + // Now we need to actually build the bundle. + var toProcess []element + busyKeys := set[string]{} + usedKeys := set[string]{} + for k := range keys { + if ss.inprogressKeys.present(k) { + busyKeys.insert(k) + continue + } + usedKeys.insert(k) + toProcess = append(toProcess, element{ + window: win, + timestamp: wm, + pane: typex.NoFiringPane(), + holdTimestamp: wm, + transform: ss.onWindowExpiration.TransformID, + family: ss.onWindowExpiration.TimerFamily, + sequence: 1, + keyBytes: []byte(k), + elmBytes: nil, + }) + } + em.addPending(len(toProcess)) + ss.watermarkHolds.Add(wm, 1) + ss.makeInProgressBundle( + func() string { return rb.BundleID }, + toProcess, + wm, + usedKeys, + map[mtime.Time]int{wm: 1}, + ) + ss.expiryWindowsByBundles[rb.BundleID] = win + + slog.Debug("OnWindowExpiration-Bundle Created", slog.Any("bundle", rb), slog.Any("usedKeys", usedKeys), slog.Any("window", win), slog.Any("toProcess", toProcess), slog.Any("busyKeys", busyKeys)) + // We're already in the refreshCond critical section. + // Insert that this is in progress here to avoid a race condition. + em.inprogressBundles.insert(rb.BundleID) + em.injectedBundles = append(em.injectedBundles, rb) + + // Remove the key accounting, or continue tracking which keys still need clearing. + if len(busyKeys) == 0 { + delete(ss.keysToExpireByWindow, win) + } else { + ss.keysToExpireByWindow[win] = busyKeys + } + } + return preventDownstreamUpdate +} + // bundleReady returns the maximum allowed watermark for this stage, and whether // it's permitted to execute by side inputs. func (ss *stageState) bundleReady(em *ElementManager, emNow mtime.Time) (mtime.Time, bool, bool) { diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager_test.go index d5904b13fb88..0d7da5ea163f 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager_test.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "io" + "sync/atomic" "testing" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" @@ -524,3 +525,162 @@ func TestElementManager(t *testing.T) { } }) } + +func TestElementManager_OnWindowExpiration(t *testing.T) { + t.Run("createOnWindowExpirationBundles", func(t *testing.T) { + // Unlike the other tests above, we synthesize the input configuration, + em := NewElementManager(Config{}) + var instID uint64 + em.nextBundID = func() string { + return fmt.Sprintf("inst%03d", atomic.AddUint64(&instID, 1)) + } + em.AddStage("impulse", nil, []string{"input"}, nil) + em.AddStage("dofn", []string{"input"}, nil, nil) + onWE := StaticTimerID{ + TransformID: "dofn1", + TimerFamily: "onWinExp", + } + em.StageOnWindowExpiration("dofn", onWE) + em.Impulse("impulse") + + stage := em.stages["dofn"] + stage.pendingByKeys = map[string]*dataAndTimers{} + stage.inprogressKeys = set[string]{} + + validateInProgressExpiredWindows := func(t *testing.T, win typex.Window, want int) { + t.Helper() + if got := stage.inProgressExpiredWindows[win]; got != want { + t.Errorf("stage.inProgressExpiredWindows[%v] = %v, want %v", win, got, want) + } + } + validateSideBundles := func(t *testing.T, keys set[string]) { + t.Helper() + if len(em.injectedBundles) == 0 { + t.Errorf("no injectedBundles exist when checking keys: %v", keys) + } + // Check that all keys are marked as in progress + for k := range keys { + if !stage.inprogressKeys.present(k) { + t.Errorf("key %q not marked as in progress", k) + } + } + + bundleID := "" + sideBundles: + for _, rb := range em.injectedBundles { + // find that a side channel bundle exists with these keys. + bkeys := stage.inprogressKeysByBundle[rb.BundleID] + if len(bkeys) != len(keys) { + continue sideBundles + } + for k := range keys { + if !bkeys.present(k) { + continue sideBundles + } + } + bundleID = rb.BundleID + break + } + if bundleID == "" { + t.Errorf("no bundle found with all the given keys: %v: bundles: %v keysByBundle: %v", keys, em.injectedBundles, stage.inprogressKeysByBundle) + } + } + + newOut := mtime.EndOfGlobalWindowTime + // No windows exist, so no side channel bundles should be set. + if got, want := stage.createOnWindowExpirationBundles(newOut, em), false; got != want { + t.Errorf("createOnWindowExpirationBundles(%v) = %v, want %v", newOut, got, want) + } + // Validate that no side channel bundles were created. + if got, want := len(stage.inProgressExpiredWindows), 0; got != want { + t.Errorf("len(stage.inProgressExpiredWindows) = %v, want %v", got, want) + } + if got, want := len(em.injectedBundles), 0; got != want { + t.Errorf("len(em.injectedBundles) = %v, want %v", got, want) + } + + // Configure a few conditions to validate in the call. + // Each window is in it's own bundle, all are in the same bundle. + // Bundle 1 + expiredWindow1 := window.IntervalWindow{Start: 0, End: newOut - 1} + + akey := "\u0004key1" + keys1 := singleSet(akey) + stage.keysToExpireByWindow[expiredWindow1] = keys1 + // Bundle 2 + expiredWindow2 := window.IntervalWindow{Start: 1, End: newOut - 1} + keys2 := singleSet("\u0004key2") + keys2.insert("\u0004key3") + keys2.insert("\u0004key4") + stage.keysToExpireByWindow[expiredWindow2] = keys2 + + // We should never see this key and window combination, as the window is + // not yet expired. + liveWindow := window.IntervalWindow{Start: 2, End: newOut + 1} + stage.keysToExpireByWindow[liveWindow] = singleSet("\u0010keyNotSeen") + + if got, want := stage.createOnWindowExpirationBundles(newOut, em), true; got != want { + t.Errorf("createOnWindowExpirationBundles(%v) = %v, want %v", newOut, got, want) + } + + // We should only see 2 injectedBundles at this point. + if got, want := len(em.injectedBundles), 2; got != want { + t.Errorf("len(em.injectedBundles) = %v, want %v", got, want) + } + + validateInProgressExpiredWindows(t, expiredWindow1, 1) + validateInProgressExpiredWindows(t, expiredWindow2, 1) + validateSideBundles(t, keys1) + validateSideBundles(t, keys2) + + // Bundle 3 + expiredWindow3 := window.IntervalWindow{Start: 3, End: newOut - 1} + keys3 := singleSet(akey) // We shouldn't see this key, since it's in progress. + keys3.insert("\u0004key5") // We should see this key since it isn't. + stage.keysToExpireByWindow[expiredWindow3] = keys3 + + if got, want := stage.createOnWindowExpirationBundles(newOut, em), true; got != want { + t.Errorf("createOnWindowExpirationBundles(%v) = %v, want %v", newOut, got, want) + } + + // We should see 3 injectedBundles at this point. + if got, want := len(em.injectedBundles), 3; got != want { + t.Errorf("len(em.injectedBundles) = %v, want %v", got, want) + } + + validateInProgressExpiredWindows(t, expiredWindow1, 1) + validateInProgressExpiredWindows(t, expiredWindow2, 1) + validateInProgressExpiredWindows(t, expiredWindow3, 1) + validateSideBundles(t, keys1) + validateSideBundles(t, keys2) + validateSideBundles(t, singleSet("\u0004key5")) + + // remove key1 from "inprogress keys", and the associated bundle. + stage.inprogressKeys.remove(akey) + delete(stage.inProgressExpiredWindows, expiredWindow1) + for bundID, bkeys := range stage.inprogressKeysByBundle { + if bkeys.present(akey) { + t.Logf("bundID: %v, bkeys: %v, keyByBundle: %v", bundID, bkeys, stage.inprogressKeysByBundle) + delete(stage.inprogressKeysByBundle, bundID) + win := stage.expiryWindowsByBundles[bundID] + delete(stage.expiryWindowsByBundles, bundID) + if win != expiredWindow1 { + t.Fatalf("Unexpected window: got %v, want %v", win, expiredWindow1) + } + break + } + } + + // Now we should get another bundle for expiredWindow3, and have none for expiredWindow1 + if got, want := stage.createOnWindowExpirationBundles(newOut, em), true; got != want { + t.Errorf("createOnWindowExpirationBundles(%v) = %v, want %v", newOut, got, want) + } + + validateInProgressExpiredWindows(t, expiredWindow1, 0) + validateInProgressExpiredWindows(t, expiredWindow2, 1) + validateInProgressExpiredWindows(t, expiredWindow3, 2) + validateSideBundles(t, keys1) // Should still have this key present, but with a different bundle. + validateSideBundles(t, keys2) + validateSideBundles(t, singleSet("\u0004key5")) // still exist.. + }) +} diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index 614edee47721..fde62f00c7c1 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -318,6 +318,10 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic if stage.stateful { em.StageStateful(stage.ID) } + if stage.onWindowExpiration.TimerFamily != "" { + slog.Debug("OnWindowExpiration", slog.String("stage", stage.ID), slog.Any("values", stage.onWindowExpiration)) + em.StageOnWindowExpiration(stage.ID, stage.onWindowExpiration) + } if len(stage.processingTimeTimers) > 0 { em.StageProcessingTimeTimers(stage.ID, stage.processingTimeTimers) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go index deef259a99d1..6158cd6d612c 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go @@ -45,6 +45,7 @@ var supportedRequirements = map[string]struct{}{ urns.RequirementSplittableDoFn: {}, urns.RequirementStatefulProcessing: {}, urns.RequirementBundleFinalization: {}, + urns.RequirementOnWindowExpiration: {}, } // TODO, move back to main package, and key off of executor handlers? diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go index a2840760bf7a..894a6e1427a2 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go @@ -182,8 +182,6 @@ func (s *Server) Prepare(ctx context.Context, req *jobpb.PrepareJobRequest) (_ * check("TimerFamilySpecs.TimeDomain.Urn", spec.GetTimeDomain(), pipepb.TimeDomain_EVENT_TIME, pipepb.TimeDomain_PROCESSING_TIME) } - check("OnWindowExpirationTimerFamily", pardo.GetOnWindowExpirationTimerFamilySpec(), "") // Unsupported for now. - // Check for a stateful SDF and direct user to https://github.com/apache/beam/issues/32139 if pardo.GetRestrictionCoderId() != "" && isStateful { check("Splittable+Stateful DoFn", "See https://github.com/apache/beam/issues/32139 for information.", "") diff --git a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go index dceaa9ab8fcb..0d3ec7c365c1 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/preprocess.go +++ b/sdks/go/pkg/beam/runners/prism/internal/preprocess.go @@ -449,6 +449,9 @@ func finalizeStage(stg *stage, comps *pipepb.Components, pipelineFacts *fusionFa if len(pardo.GetTimerFamilySpecs())+len(pardo.GetStateSpecs())+len(pardo.GetOnWindowExpirationTimerFamilySpec()) > 0 { stg.stateful = true } + if pardo.GetOnWindowExpirationTimerFamilySpec() != "" { + stg.onWindowExpiration = engine.StaticTimerID{TransformID: link.Transform, TimerFamily: pardo.GetOnWindowExpirationTimerFamilySpec()} + } sis = pardo.GetSideInputs() } if _, ok := sis[link.Local]; ok { diff --git a/sdks/go/pkg/beam/runners/prism/internal/stage.go b/sdks/go/pkg/beam/runners/prism/internal/stage.go index 9f00c22789b6..9dd6cbdafec8 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/stage.go +++ b/sdks/go/pkg/beam/runners/prism/internal/stage.go @@ -57,18 +57,20 @@ type link struct { // account, but all serialization boundaries remain since the pcollections // would continue to get serialized. type stage struct { - ID string - transforms []string - primaryInput string // PCollection used as the parallel input. - outputs []link // PCollections that must escape this stage. - sideInputs []engine.LinkID // Non-parallel input PCollections and their consumers - internalCols []string // PCollections that escape. Used for precise coder sending. - envID string - finalize bool - stateful bool + ID string + transforms []string + primaryInput string // PCollection used as the parallel input. + outputs []link // PCollections that must escape this stage. + sideInputs []engine.LinkID // Non-parallel input PCollections and their consumers + internalCols []string // PCollections that escape. Used for precise coder sending. + envID string + finalize bool + stateful bool + onWindowExpiration engine.StaticTimerID + // hasTimers indicates the transform+timerfamily pairs that need to be waited on for // the stage to be considered complete. - hasTimers []struct{ Transform, TimerFamily string } + hasTimers []engine.StaticTimerID processingTimeTimers map[string]bool exe transformExecuter @@ -452,7 +454,7 @@ func buildDescriptor(stg *stage, comps *pipepb.Components, wk *worker.W, em *eng } } for timerID, v := range pardo.GetTimerFamilySpecs() { - stg.hasTimers = append(stg.hasTimers, struct{ Transform, TimerFamily string }{Transform: tid, TimerFamily: timerID}) + stg.hasTimers = append(stg.hasTimers, engine.StaticTimerID{TransformID: tid, TimerFamily: timerID}) if v.TimeDomain == pipepb.TimeDomain_PROCESSING_TIME { if stg.processingTimeTimers == nil { stg.processingTimeTimers = map[string]bool{} diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go index 83ad1bda9841..14cd84aef821 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/bundle.go @@ -42,7 +42,7 @@ type B struct { InputTransformID string Input []*engine.Block // Data and Timers for this bundle. EstimatedInputElements int - HasTimers []struct{ Transform, TimerFamily string } // Timer streams to terminate. + HasTimers []engine.StaticTimerID // Timer streams to terminate. // IterableSideInputData is a map from transformID + inputID, to window, to data. IterableSideInputData map[SideInputKey]map[typex.Window][][]byte @@ -190,7 +190,7 @@ func (b *B) ProcessOn(ctx context.Context, wk *W) <-chan struct{} { for _, tid := range b.HasTimers { timers = append(timers, &fnpb.Elements_Timers{ InstructionId: b.InstID, - TransformId: tid.Transform, + TransformId: tid.TransformID, TimerFamilyId: tid.TimerFamily, IsLast: true, }) From 02f9cb94fcd31527933164ecc3709c5d9f04cae3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 12 Dec 2024 22:16:26 -0800 Subject: [PATCH 27/33] Bump google.golang.org/api from 0.210.0 to 0.211.0 in /sdks (#33374) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.210.0 to 0.211.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.210.0...v0.211.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 8 ++++---- sdks/go.sum | 16 ++++++++-------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e6a182035f5e..79b32b051df3 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -58,7 +58,7 @@ require ( golang.org/x/sync v0.10.0 golang.org/x/sys v0.28.0 golang.org/x/text v0.21.0 - google.golang.org/api v0.210.0 + google.golang.org/api v0.211.0 google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 google.golang.org/grpc v1.67.2 google.golang.org/protobuf v1.35.2 @@ -75,7 +75,7 @@ require ( require ( cel.dev/expr v0.16.1 // indirect - cloud.google.com/go/auth v0.11.0 // indirect + cloud.google.com/go/auth v0.12.1 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.6 // indirect cloud.google.com/go/monitoring v1.21.2 // indirect dario.cat/mergo v1.0.0 // indirect @@ -194,6 +194,6 @@ require ( golang.org/x/mod v0.20.0 // indirect golang.org/x/tools v0.24.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241113202542-65e8d215514f // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241206012308-a4fef0638583 // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 34e34ef2f8cd..0e7792f499f4 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -101,8 +101,8 @@ cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVo cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= -cloud.google.com/go/auth v0.11.0 h1:Ic5SZz2lsvbYcWT5dfjNWgw6tTlGi2Wc8hyQSC9BstA= -cloud.google.com/go/auth v0.11.0/go.mod h1:xxA5AqpDrvS+Gkmo9RqrGGRh6WSNKKOXhY3zNOr38tI= +cloud.google.com/go/auth v0.12.1 h1:n2Bj25BUMM0nvE9D2XLTiImanwZhO3DkfWSYS/SAJP4= +cloud.google.com/go/auth v0.12.1/go.mod h1:BFMu+TNpF3DmvfBO9ClqTR/SiqVIm7LukKF9mbendF4= cloud.google.com/go/auth/oauth2adapt v0.2.6 h1:V6a6XDu2lTwPZWOawrAa9HUK+DB2zfJyTuciBG5hFkU= cloud.google.com/go/auth/oauth2adapt v0.2.6/go.mod h1:AlmsELtlEBnaNTL7jCj8VQFLy6mbZv0s4Q7NGBeQ5E8= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= @@ -1707,8 +1707,8 @@ google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/ google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= -google.golang.org/api v0.210.0 h1:HMNffZ57OoZCRYSbdWVRoqOa8V8NIHLL0CzdBPLztWk= -google.golang.org/api v0.210.0/go.mod h1:B9XDZGnx2NtyjzVkOVTGrFSAVZgPcbedzKg/gTLwqBs= +google.golang.org/api v0.211.0 h1:IUpLjq09jxBSV1lACO33CGY3jsRcbctfGzhj+ZSE/Bg= +google.golang.org/api v0.211.0/go.mod h1:XOloB4MXFH4UTlQSGuNUxw0UT74qdENK8d6JNsXKLi0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1850,10 +1850,10 @@ google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20241118233622-e639e219e697 h1:ToEetK57OidYuqD4Q5w+vfEnPvPpuTwedCNVohYJfNk= google.golang.org/genproto v0.0.0-20241118233622-e639e219e697/go.mod h1:JJrvXBWRZaFMxBufik1a4RpFw4HhgVtBBWQeQgUj2cc= -google.golang.org/genproto/googleapis/api v0.0.0-20241113202542-65e8d215514f h1:M65LEviCfuZTfrfzwwEoxVtgvfkFkBUbFnRbxCXuXhU= -google.golang.org/genproto/googleapis/api v0.0.0-20241113202542-65e8d215514f/go.mod h1:Yo94eF2nj7igQt+TiJ49KxjIH8ndLYPZMIRSiRcEbg0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697 h1:LWZqQOEjDyONlF1H6afSWpAL/znlREo2tHfLoe+8LMA= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697 h1:pgr/4QbFyktUv9CtQ/Fq4gzEE6/Xs7iCXbktaGzLHbQ= +google.golang.org/genproto/googleapis/api v0.0.0-20241118233622-e639e219e697/go.mod h1:+D9ySVjN8nY8YCVjc5O7PZDIdZporIDY3KaGfJunh88= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241206012308-a4fef0638583 h1:IfdSdTcLFy4lqUQrQJLkLt1PB+AsqVz6lwkWPzWEz10= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241206012308-a4fef0638583/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 994d2f0cad0a4e7d2511c518af6d4f5a9696cd21 Mon Sep 17 00:00:00 2001 From: "Doroszlai, Attila" <6454655+adoroszlai@users.noreply.github.com> Date: Fri, 13 Dec 2024 15:13:51 +0100 Subject: [PATCH 28/33] Fix Apache snapshot repo usage (#33370) * limit Apache snapshot repo content type * move up Confluent repo --- .../apache/beam/gradle/Repositories.groovy | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/Repositories.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/Repositories.groovy index 52cbbd15c35b..58ec64a0add3 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/Repositories.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/Repositories.groovy @@ -39,20 +39,25 @@ class Repositories { mavenCentral() mavenLocal() + // For Confluent Kafka dependencies + maven { + url "https://packages.confluent.io/maven/" + content { includeGroup "io.confluent" } + } + // Release staging repository maven { url "https://oss.sonatype.org/content/repositories/staging/" } // Apache nightly snapshots - maven { url "https://repository.apache.org/snapshots" } + maven { + url "https://repository.apache.org/snapshots" + mavenContent { + snapshotsOnly() + } + } // Apache release snapshots maven { url "https://repository.apache.org/content/repositories/releases" } - - // For Confluent Kafka dependencies - maven { - url "https://packages.confluent.io/maven/" - content { includeGroup "io.confluent" } - } } // Apply a plugin which provides the 'updateOfflineRepository' task that creates an offline From a6061feb05363a6175e28f80f684b6d306a42150 Mon Sep 17 00:00:00 2001 From: twosom <72733442+twosom@users.noreply.github.com> Date: Sat, 14 Dec 2024 00:44:48 +0900 Subject: [PATCH 29/33] Batch optimized SparkRunner groupByKey (#33322) * feat : optimized SparkRunner batch groupByKey * update CHANGES.md * touch trigger files * remove unused test --- .../beam_PostCommit_Java_PVR_Spark_Batch.json | 2 +- ...PostCommit_Java_ValidatesRunner_Spark.json | 3 +- ...mit_Java_ValidatesRunner_Spark_Java11.json | 3 +- CHANGES.md | 1 + .../translation/GroupCombineFunctions.java | 46 ++++++--- .../GroupNonMergingWindowsFunctions.java | 98 ++++++++++++++----- .../GroupNonMergingWindowsFunctionsTest.java | 57 ----------- 7 files changed, 116 insertions(+), 94 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Java_PVR_Spark_Batch.json b/.github/trigger_files/beam_PostCommit_Java_PVR_Spark_Batch.json index f1ba03a243ee..455144f02a35 100644 --- a/.github/trigger_files/beam_PostCommit_Java_PVR_Spark_Batch.json +++ b/.github/trigger_files/beam_PostCommit_Java_PVR_Spark_Batch.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 5 + "modification": 6 } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json index 9b023f630c36..03d86a8d023e 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json @@ -2,5 +2,6 @@ "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", "https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test", - "https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test" + "https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test", + "https://github.com/apache/beam/pull/33322": "noting that PR #33322 should run this test" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json index 9b023f630c36..03d86a8d023e 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.json @@ -2,5 +2,6 @@ "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", "https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test", - "https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test" + "https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test", + "https://github.com/apache/beam/pull/33322": "noting that PR #33322 should run this test" } diff --git a/CHANGES.md b/CHANGES.md index ea29b6aadc7d..5bbb10f76efb 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,7 @@ ## New Features / Improvements +* Improved batch performance of SparkRunner's GroupByKey ([#20943](https://github.com/apache/beam/pull/20943)). * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Support OnWindowExpiration in Prism ([#32211](https://github.com/apache/beam/issues/32211)). * This enables initial Java GroupIntoBatches support. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java index 62c5e2579427..1d8901ed5ffc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.spark.translation; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.util.ByteArray; import org.apache.beam.sdk.coders.Coder; @@ -27,6 +30,7 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -49,18 +53,36 @@ public static JavaRDD>>> groupByKeyOnly( @Nullable Partitioner partitioner) { // we use coders to convert objects in the PCollection to byte arrays, so they // can be transferred over the network for the shuffle. - JavaPairRDD pairRDD = - rdd.map(new ReifyTimestampsAndWindowsFunction<>()) - .mapToPair(TranslationUtils.toPairFunction()) - .mapToPair(CoderHelpers.toByteFunction(keyCoder, wvCoder)); - - // If no partitioner is passed, the default group by key operation is called - JavaPairRDD> groupedRDD = - (partitioner != null) ? pairRDD.groupByKey(partitioner) : pairRDD.groupByKey(); - - return groupedRDD - .mapToPair(CoderHelpers.fromByteFunctionIterable(keyCoder, wvCoder)) - .map(new TranslationUtils.FromPairFunction<>()); + final JavaPairRDD pairRDD = + rdd.mapPartitionsToPair( + (Iterator>> iter) -> + Iterators.transform( + iter, + (WindowedValue> wv) -> { + final K key = wv.getValue().getKey(); + final WindowedValue windowedValue = wv.withValue(wv.getValue().getValue()); + final ByteArray keyBytes = + new ByteArray(CoderHelpers.toByteArray(key, keyCoder)); + final byte[] windowedValueBytes = + CoderHelpers.toByteArray(windowedValue, wvCoder); + return Tuple2.apply(keyBytes, windowedValueBytes); + })); + + final JavaPairRDD> combined = + GroupNonMergingWindowsFunctions.combineByKey(pairRDD, partitioner).cache(); + + return combined.mapPartitions( + (Iterator>> iter) -> + Iterators.transform( + iter, + (Tuple2> tuple) -> { + final K key = CoderHelpers.fromByteArray(tuple._1().getValue(), keyCoder); + final List> windowedValues = + tuple._2().stream() + .map(bytes -> CoderHelpers.fromByteArray(bytes, wvCoder)) + .collect(Collectors.toList()); + return KV.of(key, windowedValues); + })); } /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java index 2461d5cc8d66..14630fbb0a1f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java @@ -17,7 +17,9 @@ */ package org.apache.beam.runners.spark.translation; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.Objects; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.util.ByteArray; @@ -41,6 +43,9 @@ import org.apache.spark.Partitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -259,9 +264,12 @@ private WindowedValue> decodeItem(Tuple2 item) { } /** - * Group all values with a given key for that composite key with Spark's groupByKey, dropping the - * Window (which must be GlobalWindow) and returning the grouped result in the appropriate global - * window. + * Groups values with a given key using Spark's combineByKey operation in the Global Window + * context. The window information (which must be GlobalWindow) is dropped during processing, and + * the grouped results are returned in the appropriate global window with the maximum timestamp. + * + *

This implementation uses {@link JavaPairRDD#combineByKey} for better performance compared to + * {@link JavaPairRDD#groupByKey}, as it allows for local aggregation before shuffle operations. */ static JavaRDD>>> groupByKeyInGlobalWindow( @@ -269,24 +277,70 @@ JavaRDD>>> groupByKeyInGlobalWindow( Coder keyCoder, Coder valueCoder, Partitioner partitioner) { - JavaPairRDD rawKeyValues = - rdd.mapToPair( - wv -> - new Tuple2<>( - new ByteArray(CoderHelpers.toByteArray(wv.getValue().getKey(), keyCoder)), - CoderHelpers.toByteArray(wv.getValue().getValue(), valueCoder))); - - JavaPairRDD> grouped = - (partitioner == null) ? rawKeyValues.groupByKey() : rawKeyValues.groupByKey(partitioner); - return grouped.map( - kvs -> - WindowedValue.timestampedValueInGlobalWindow( - KV.of( - CoderHelpers.fromByteArray(kvs._1.getValue(), keyCoder), - Iterables.transform( - kvs._2, - encodedValue -> CoderHelpers.fromByteArray(encodedValue, valueCoder))), - GlobalWindow.INSTANCE.maxTimestamp(), - PaneInfo.ON_TIME_AND_ONLY_FIRING)); + final JavaPairRDD rawKeyValues = + rdd.mapPartitionsToPair( + (Iterator>> iter) -> + Iterators.transform( + iter, + (WindowedValue> wv) -> { + final ByteArray keyBytes = + new ByteArray(CoderHelpers.toByteArray(wv.getValue().getKey(), keyCoder)); + final byte[] valueBytes = + CoderHelpers.toByteArray(wv.getValue().getValue(), valueCoder); + return Tuple2.apply(keyBytes, valueBytes); + })); + + JavaPairRDD> combined = combineByKey(rawKeyValues, partitioner).cache(); + + return combined.mapPartitions( + (Iterator>> iter) -> + Iterators.transform( + iter, + kvs -> + WindowedValue.timestampedValueInGlobalWindow( + KV.of( + CoderHelpers.fromByteArray(kvs._1.getValue(), keyCoder), + Iterables.transform( + kvs._2(), + encodedValue -> + CoderHelpers.fromByteArray(encodedValue, valueCoder))), + GlobalWindow.INSTANCE.maxTimestamp(), + PaneInfo.ON_TIME_AND_ONLY_FIRING))); + } + + /** + * Combines values by key using Spark's {@link JavaPairRDD#combineByKey} operation. + * + * @param rawKeyValues Input RDD of key-value pairs + * @param partitioner Optional custom partitioner for data distribution + * @return RDD with values combined into Lists per key + */ + static JavaPairRDD> combineByKey( + JavaPairRDD rawKeyValues, @Nullable Partitioner partitioner) { + + final Function> createCombiner = + value -> { + List list = new ArrayList<>(); + list.add(value); + return list; + }; + + final Function2, byte[], List> mergeValues = + (list, value) -> { + list.add(value); + return list; + }; + + final Function2, List, List> mergeCombiners = + (list1, list2) -> { + list1.addAll(list2); + return list1; + }; + + if (partitioner == null) { + return rawKeyValues.combineByKey(createCombiner, mergeValues, mergeCombiners); + } + + return rawKeyValues.combineByKey(createCombiner, mergeValues, mergeCombiners, partitioner); } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java index ed7bc078564e..fd299924af91 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java @@ -18,12 +18,6 @@ package org.apache.beam.runners.spark.translation; import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; import java.util.Arrays; import java.util.Iterator; @@ -45,9 +39,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Bytes; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; @@ -121,54 +112,6 @@ public void testGbkIteratorValuesCannotBeReiterated() throws Coder.NonDeterminis } } - @Test - @SuppressWarnings({"rawtypes", "unchecked"}) - public void testGroupByKeyInGlobalWindowWithPartitioner() { - // mocking - Partitioner mockPartitioner = mock(Partitioner.class); - JavaRDD mockRdd = mock(JavaRDD.class); - Coder mockKeyCoder = mock(Coder.class); - Coder mockValueCoder = mock(Coder.class); - JavaPairRDD mockRawKeyValues = mock(JavaPairRDD.class); - JavaPairRDD mockGrouped = mock(JavaPairRDD.class); - - when(mockRdd.mapToPair(any())).thenReturn(mockRawKeyValues); - when(mockRawKeyValues.groupByKey(any(Partitioner.class))) - .thenAnswer( - invocation -> { - Partitioner partitioner = invocation.getArgument(0); - assertEquals(partitioner, mockPartitioner); - return mockGrouped; - }); - when(mockGrouped.map(any())).thenReturn(mock(JavaRDD.class)); - - GroupNonMergingWindowsFunctions.groupByKeyInGlobalWindow( - mockRdd, mockKeyCoder, mockValueCoder, mockPartitioner); - - verify(mockRawKeyValues, never()).groupByKey(); - verify(mockRawKeyValues, times(1)).groupByKey(any(Partitioner.class)); - } - - @Test - @SuppressWarnings({"rawtypes", "unchecked"}) - public void testGroupByKeyInGlobalWindowWithoutPartitioner() { - // mocking - JavaRDD mockRdd = mock(JavaRDD.class); - Coder mockKeyCoder = mock(Coder.class); - Coder mockValueCoder = mock(Coder.class); - JavaPairRDD mockRawKeyValues = mock(JavaPairRDD.class); - JavaPairRDD mockGrouped = mock(JavaPairRDD.class); - - when(mockRdd.mapToPair(any())).thenReturn(mockRawKeyValues); - when(mockRawKeyValues.groupByKey()).thenReturn(mockGrouped); - - GroupNonMergingWindowsFunctions.groupByKeyInGlobalWindow( - mockRdd, mockKeyCoder, mockValueCoder, null); - - verify(mockRawKeyValues, times(1)).groupByKey(); - verify(mockRawKeyValues, never()).groupByKey(any(Partitioner.class)); - } - private GroupByKeyIterator createGbkIterator() throws Coder.NonDeterministicException { return createGbkIterator( From 011ec94b9e36bb013f165f9dc4799474aeaa5ac2 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Fri, 13 Dec 2024 08:46:43 -0800 Subject: [PATCH 30/33] remove flaky tests that terminate before read. (#33371) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../beam/core/runtime/harness/datamgr_test.go | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go index 92c4d0a8f8cd..9f6f8a986a3f 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go +++ b/sdks/go/pkg/beam/core/runtime/harness/datamgr_test.go @@ -261,22 +261,6 @@ func TestElementChan(t *testing.T) { return elms }, wantSum: 6, wantCount: 3, - }, { - name: "FillBufferThenAbortThenRead", - sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - for i := 0; i < bufElements+2; i++ { - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(1, false)}}) - } - elms := openChan(ctx, t, c, timerID) - c.removeInstruction(instID) - - // These will be ignored - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(1, false)}}) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(2, false)}}) - client.Send(&fnpb.Elements{Data: []*fnpb.Elements_Data{dataElm(3, true)}}) - return elms - }, - wantSum: bufElements, wantCount: bufElements, }, { name: "DataThenReaderThenLast", sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { @@ -389,18 +373,6 @@ func TestElementChan(t *testing.T) { return elms }, wantSum: 0, wantCount: 0, - }, { - name: "SomeTimersAndADataThenReaderThenCleanup", - sequenceFn: func(ctx context.Context, t *testing.T, client *fakeChanClient, c *DataChannel) <-chan exec.Elements { - client.Send(&fnpb.Elements{ - Timers: []*fnpb.Elements_Timers{timerElm(1, false), timerElm(2, true)}, - Data: []*fnpb.Elements_Data{dataElm(3, true)}, - }) - elms := openChan(ctx, t, c, timerID) - c.removeInstruction(instID) - return elms - }, - wantSum: 6, wantCount: 3, }, } for _, test := range tests { From f92dde16fa66fe104cd6057612bfadb4732c186b Mon Sep 17 00:00:00 2001 From: Damon Date: Fri, 13 Dec 2024 10:41:07 -0800 Subject: [PATCH 31/33] Clean up Java Tests GitHub workflow (#33354) * Remove use of static credentials * Stage for adding back dataflow * Remove unnecessary dataflow test --- .github/workflows/java_tests.yml | 72 +------------------------------- 1 file changed, 2 insertions(+), 70 deletions(-) diff --git a/.github/workflows/java_tests.yml b/.github/workflows/java_tests.yml index 1d6441b24681..bdc78b88cb97 100644 --- a/.github/workflows/java_tests.yml +++ b/.github/workflows/java_tests.yml @@ -20,11 +20,7 @@ name: Java Tests on: workflow_dispatch: - inputs: - runDataflow: - description: 'Type "true" if you want to run Dataflow tests (GCP variables must be configured, check CI.md)' - default: 'false' - required: false + schedule: - cron: '10 2 * * *' push: @@ -33,8 +29,7 @@ on: pull_request: branches: ['master', 'release-*'] tags: ['v*'] - paths: ['sdks/java/**', 'model/**', 'runners/**', 'examples/java/**', - 'examples/kotlin/**', 'release/**', 'buildSrc/**'] + paths: ['sdks/java/**', 'model/**', 'runners/**', 'examples/java/**', 'examples/kotlin/**', 'release/**', 'buildSrc/**'] # This allows a subsequently queued workflow run to interrupt previous runs concurrency: group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login}}' @@ -44,26 +39,6 @@ env: GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} jobs: - check_gcp_variables: - timeout-minutes: 5 - name: "Check GCP variables set" - runs-on: [self-hosted, ubuntu-20.04, main] - outputs: - gcp-variables-set: ${{ steps.check_gcp_variables.outputs.gcp-variables-set }} - steps: - - name: Check out code - uses: actions/checkout@v4 - - name: "Check are GCP variables set" - run: "./scripts/ci/ci_check_are_gcp_variables_set.sh" - id: check_gcp_variables - env: - GCP_PROJECT_ID: ${{ secrets.GCP_PROJECT_ID }} - GCP_SA_EMAIL: ${{ secrets.GCP_SA_EMAIL }} - GCP_SA_KEY: ${{ secrets.GCP_SA_KEY }} - GCP_TESTING_BUCKET: ${{ secrets.GCP_TESTING_BUCKET }} - GCP_REGION: "not-needed-here" - GCP_PYTHON_WHEELS_BUCKET: "not-needed-here" - java_unit_tests: name: 'Java Unit Tests' runs-on: ${{ matrix.os }} @@ -152,46 +127,3 @@ jobs: with: name: java_wordcount_direct_runner-${{matrix.os}} path: examples/java/build/reports/tests/integrationTest - - java_wordcount_dataflow: - name: 'Java Wordcount Dataflow' - needs: - - check_gcp_variables - runs-on: ${{ matrix.os }} - strategy: - fail-fast: false - matrix: - os: [[self-hosted, ubuntu-20.04, main], windows-latest] - # TODO(https://github.com/apache/beam/issues/31848) run on Dataflow after fixes credential on macOS/win GHA runner - if: | - needs.check_gcp_variables.outputs.gcp-variables-set == 'true' && - (github.event_name == 'workflow_dispatch' && github.event.inputs.runDataflow == 'true') - steps: - - name: Check out code - uses: actions/checkout@v4 - with: - persist-credentials: false - submodules: recursive - - name: Setup environment - uses: ./.github/actions/setup-environment-action - with: - java-version: 11 - go-version: default - - name: Authenticate on GCP - uses: google-github-actions/auth@v1 - with: - credentials_json: ${{ secrets.GCP_SA_KEY }} - project_id: ${{ secrets.GCP_PROJECT_ID }} - - name: Run WordCount - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: integrationTest - arguments: -p examples/ --tests org.apache.beam.examples.WordCountIT - -DintegrationTestPipelineOptions=[\"--runner=DataflowRunner\",\"--project=${{ secrets.GCP_PROJECT_ID }}\",\"--tempRoot=gs://${{ secrets.GCP_TESTING_BUCKET }}/tmp/\"] - -DintegrationTestRunner=dataflow - - name: Upload test logs - uses: actions/upload-artifact@v4 - if: always() - with: - name: java_wordcount_dataflow-${{matrix.os}} - path: examples/java/build/reports/tests/integrationTest \ No newline at end of file From 88398f33ce663748d87a7df61fe5d948e6e1b4f9 Mon Sep 17 00:00:00 2001 From: Jeffrey Kinard Date: Fri, 13 Dec 2024 14:41:02 -0500 Subject: [PATCH 32/33] fix BoundedTrieData get_result return type hint Signed-off-by: Jeffrey Kinard --- sdks/python/apache_beam/metrics/cells.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py index 9a62cae14691..b765c830f69c 100644 --- a/sdks/python/apache_beam/metrics/cells.py +++ b/sdks/python/apache_beam/metrics/cells.py @@ -826,7 +826,7 @@ def __repr__(self) -> str: def get_cumulative(self) -> "BoundedTrieData": return copy.deepcopy(self) - def get_result(self) -> set[tuple]: + def get_result(self) -> Set[tuple]: if self._root is None: if self._singleton is None: return set() From 2aab9cd9d1bcbcce49113423adbd526bf2392ab4 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Fri, 13 Dec 2024 20:31:09 -0500 Subject: [PATCH 33/33] Fix custom coders not being used in Reshuffle (non global window) (#33363) * Fix typehint in ReshufflePerKey on global window setting. * Only update the type hint on global window setting. Need more work in non-global windows. * Apply yapf * Fix some failed tests. * Revert change to setup.py * Fix custom coders not being used in reshuffle in non-global windows * Revert changes in setup.py. Reformat. * Make WindowedValue a generic class. Support its conversion to the correct type constraint in Beam. * Cython does not support Python generic class. Add a subclass as a workroundand keep it un-cythonized. * Add comments * Fix type error. * Remove the base class of WindowedValue in TypedWindowedValue. * Move TypedWindowedValue out from windowed_value.py * Revise the comments * Fix the module location when matching. * Fix test failure where __name__ of a type alias not found in python 3.9 * Add a note about the window coder. --------- Co-authored-by: Robert Bradshaw --- sdks/python/apache_beam/coders/coders.py | 8 +++ sdks/python/apache_beam/coders/typecoders.py | 2 + sdks/python/apache_beam/transforms/util.py | 6 +-- .../apache_beam/transforms/util_test.py | 51 ++++++++++++------- .../typehints/native_type_compatibility.py | 26 ++++++++++ .../python/apache_beam/typehints/typehints.py | 9 ++++ 6 files changed, 81 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index a0c55da81800..724f268a8312 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -1402,6 +1402,14 @@ def __hash__(self): return hash( (self.wrapped_value_coder, self.timestamp_coder, self.window_coder)) + @classmethod + def from_type_hint(cls, typehint, registry): + # type: (Any, CoderRegistry) -> WindowedValueCoder + # Ideally this'd take two parameters so that one could hint at + # the window type as well instead of falling back to the + # pickle coders. + return cls(registry.get_coder(typehint.inner_type)) + Coder.register_structured_urn( common_urns.coders.WINDOWED_VALUE.urn, WindowedValueCoder) diff --git a/sdks/python/apache_beam/coders/typecoders.py b/sdks/python/apache_beam/coders/typecoders.py index 1667cb7a916a..892f508d0136 100644 --- a/sdks/python/apache_beam/coders/typecoders.py +++ b/sdks/python/apache_beam/coders/typecoders.py @@ -94,6 +94,8 @@ def register_standard_coders(self, fallback_coder): self._register_coder_internal(str, coders.StrUtf8Coder) self._register_coder_internal(typehints.TupleConstraint, coders.TupleCoder) self._register_coder_internal(typehints.DictConstraint, coders.MapCoder) + self._register_coder_internal( + typehints.WindowedTypeConstraint, coders.WindowedValueCoder) # Default fallback coders applied in that order until the first matching # coder found. default_fallback_coders = [ diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index 43d4a6c20e94..c9fd2c76b0db 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -74,6 +74,7 @@ from apache_beam.transforms.window import TimestampedValue from apache_beam.typehints import trivial_inference from apache_beam.typehints.decorators import get_signature +from apache_beam.typehints.native_type_compatibility import TypedWindowedValue from apache_beam.typehints.sharded_key_type import ShardedKeyType from apache_beam.utils import shared from apache_beam.utils import windowed_value @@ -972,9 +973,8 @@ def restore_timestamps(element): key, windowed_values = element return [wv.with_value((key, wv.value)) for wv in windowed_values] - # TODO(https://github.com/apache/beam/issues/33356): Support reshuffling - # unpicklable objects with a non-global window setting. - ungrouped = pcoll | Map(reify_timestamps).with_output_types(Any) + ungrouped = pcoll | Map(reify_timestamps).with_input_types( + Tuple[K, V]).with_output_types(Tuple[K, TypedWindowedValue[V]]) # TODO(https://github.com/apache/beam/issues/19785) Using global window as # one of the standard window. This is to mitigate the Dataflow Java Runner diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py index 7f166f78ef0a..db73310dfe25 100644 --- a/sdks/python/apache_beam/transforms/util_test.py +++ b/sdks/python/apache_beam/transforms/util_test.py @@ -1010,32 +1010,33 @@ def format_with_timestamp(element, timestamp=beam.DoFn.TimestampParam): equal_to(expected_data), label="formatted_after_reshuffle") - def test_reshuffle_unpicklable_in_global_window(self): - global _Unpicklable + global _Unpicklable + global _UnpicklableCoder - class _Unpicklable(object): - def __init__(self, value): - self.value = value + class _Unpicklable(object): + def __init__(self, value): + self.value = value - def __getstate__(self): - raise NotImplementedError() + def __getstate__(self): + raise NotImplementedError() - def __setstate__(self, state): - raise NotImplementedError() + def __setstate__(self, state): + raise NotImplementedError() - class _UnpicklableCoder(beam.coders.Coder): - def encode(self, value): - return str(value.value).encode() + class _UnpicklableCoder(beam.coders.Coder): + def encode(self, value): + return str(value.value).encode() - def decode(self, encoded): - return _Unpicklable(int(encoded.decode())) + def decode(self, encoded): + return _Unpicklable(int(encoded.decode())) - def to_type_hint(self): - return _Unpicklable + def to_type_hint(self): + return _Unpicklable - def is_deterministic(self): - return True + def is_deterministic(self): + return True + def test_reshuffle_unpicklable_in_global_window(self): beam.coders.registry.register_coder(_Unpicklable, _UnpicklableCoder) with TestPipeline() as pipeline: @@ -1049,6 +1050,20 @@ def is_deterministic(self): | beam.Map(lambda u: u.value * 10)) assert_that(result, equal_to(expected_data)) + def test_reshuffle_unpicklable_in_non_global_window(self): + beam.coders.registry.register_coder(_Unpicklable, _UnpicklableCoder) + + with TestPipeline() as pipeline: + data = [_Unpicklable(i) for i in range(5)] + expected_data = [0, 0, 0, 10, 10, 10, 20, 20, 20, 30, 30, 30, 40, 40, 40] + result = ( + pipeline + | beam.Create(data) + | beam.WindowInto(window.SlidingWindows(size=3, period=1)) + | beam.Reshuffle() + | beam.Map(lambda u: u.value * 10)) + assert_that(result, equal_to(expected_data)) + class WithKeysTest(unittest.TestCase): def setUp(self): diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py index 6f704b37a969..381d4f7aae2b 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -24,9 +24,13 @@ import sys import types import typing +from typing import Generic +from typing import TypeVar from apache_beam.typehints import typehints +T = TypeVar('T') + _LOGGER = logging.getLogger(__name__) # Describes an entry in the type map in convert_to_beam_type. @@ -216,6 +220,18 @@ def convert_collections_to_typing(typ): return typ +# During type inference of WindowedValue, we need to pass in the inner value +# type. This cannot be achieved immediately with WindowedValue class because it +# is not parameterized. Changing it to a generic class (e.g. WindowedValue[T]) +# could work in theory. However, the class is cythonized and it seems that +# cython does not handle generic classes well. +# The workaround here is to create a separate class solely for the type +# inference purpose. This class should never be used for creating instances. +class TypedWindowedValue(Generic[T]): + def __init__(self, *args, **kwargs): + raise NotImplementedError("This class is solely for type inference") + + def convert_to_beam_type(typ): """Convert a given typing type to a Beam type. @@ -267,6 +283,12 @@ def convert_to_beam_type(typ): # TODO(https://github.com/apache/beam/issues/20076): Currently unhandled. _LOGGER.info('Converting NewType type hint to Any: "%s"', typ) return typehints.Any + elif typ_module == 'apache_beam.typehints.native_type_compatibility' and \ + getattr(typ, "__name__", typ.__origin__.__name__) == 'TypedWindowedValue': + # Need to pass through WindowedValue class so that it can be converted + # to the correct type constraint in Beam + # This is needed to fix https://github.com/apache/beam/issues/33356 + pass elif (typ_module != 'typing') and (typ_module != 'collections.abc'): # Only translate types from the typing and collections.abc modules. return typ @@ -324,6 +346,10 @@ def convert_to_beam_type(typ): match=_match_is_exactly_collection, arity=1, beam_type=typehints.Collection), + _TypeMapEntry( + match=_match_issubclass(TypedWindowedValue), + arity=1, + beam_type=typehints.WindowedValue), ] # Find the first matching entry. diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 0e18e887c2a0..a65a0f753826 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -1213,6 +1213,15 @@ def type_check(self, instance): repr(self.inner_type), instance.value.__class__.__name__)) + def bind_type_variables(self, bindings): + bound_inner_type = bind_type_variables(self.inner_type, bindings) + if bound_inner_type == self.inner_type: + return self + return WindowedValue[bound_inner_type] + + def __repr__(self): + return 'WindowedValue[%s]' % repr(self.inner_type) + class GeneratorHint(IteratorHint): """A Generator type hint.