From 6bbe0bd6a5e7a25cb4961700db12e9ec24470010 Mon Sep 17 00:00:00 2001 From: Talat Uyarer Date: Tue, 18 Mar 2025 10:57:12 -0700 Subject: [PATCH 01/48] Kafka Consumer Group Properties --- .../meta/provider/kafka/BeamKafkaTable.java | 4 ++ .../provider/kafka/KafkaTableProvider.java | 33 +++++++++-- .../kafka/KafkaTableProviderTest.java | 56 ++++++++++++++++--- 3 files changed, 79 insertions(+), 14 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java index ab1817f6d75c..d7fdbfe3505a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java @@ -161,6 +161,10 @@ public List getTopics() { return topics; } + public Map getConfigUpdates() { + return configUpdates; + } + @Override public BeamTableStatistics getTableStatistics(PipelineOptions options) { if (rowCountStatistics == null) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java index 06cc4ea0beff..6dd6d13428de 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java @@ -21,10 +21,13 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.auto.service.AutoService; +import java.util.HashMap; import java.util.List; +import java.util.Map.Entry; import java.util.Optional; import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; @@ -112,6 +115,8 @@ public BeamSqlTable buildBeamSqlTable(Table table) { properties.has("format") ? Optional.of(properties.get("format").asText()) : Optional.empty(); + + BeamKafkaTable kafkaTable = null; if (Schemas.isNestedSchema(schema)) { Optional serializer = payloadFormat.map( @@ -120,7 +125,7 @@ public BeamSqlTable buildBeamSqlTable(Table table) { format, checkArgumentNotNull(schema.getField(PAYLOAD_FIELD).getType().getRowSchema()), TableUtils.convertNode2Map(properties))); - return new NestedPayloadKafkaTable(schema, bootstrapServers, topics, serializer); + kafkaTable = new NestedPayloadKafkaTable(schema, bootstrapServers, topics, serializer); } else { /* * CSV is handled separately because multiple rows can be produced from a single message, which @@ -129,13 +134,29 @@ public BeamSqlTable buildBeamSqlTable(Table table) { * rows. */ if (payloadFormat.orElse("csv").equals("csv")) { - return new BeamKafkaCSVTable(schema, bootstrapServers, topics); + kafkaTable = new BeamKafkaCSVTable(schema, bootstrapServers, topics); + } else { + PayloadSerializer serializer = + PayloadSerializers.getSerializer( + payloadFormat.get(), schema, TableUtils.convertNode2Map(properties)); + kafkaTable = new PayloadSerializerKafkaTable(schema, bootstrapServers, topics, serializer); + } + } + + // Get Consumer Properties from Table properties + HashMap configUpdates = new HashMap(); + while (properties.fields().hasNext()) { + Entry field = properties.fields().next(); + if (field.getKey().startsWith("properties.")) { + configUpdates.put(field.getKey(), field.getValue().textValue()); } - PayloadSerializer serializer = - PayloadSerializers.getSerializer( - payloadFormat.get(), schema, TableUtils.convertNode2Map(properties)); - return new PayloadSerializerKafkaTable(schema, bootstrapServers, topics, serializer); } + + if (!configUpdates.isEmpty()) { + kafkaTable.updateConsumerProperties(configUpdates); + } + + return kafkaTable; } @Override diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java index 09211ba06ba0..a5a6e7c55a65 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import java.util.List; +import java.util.Map; import org.apache.beam.sdk.extensions.protobuf.PayloadMessages; import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; @@ -31,6 +32,7 @@ import org.apache.beam.sdk.io.thrift.payloads.SimpleThriftMessage; import org.apache.beam.sdk.schemas.Schema; 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.ImmutableMap; import org.apache.thrift.TBase; import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.protocol.TProtocolFactory; @@ -84,6 +86,32 @@ public void testBuildWithExtraTopics() { assertEquals(ImmutableList.of(LOCATION_TOPIC, "topic2", "topic3"), kafkaTable.getTopics()); } + @Test + public void testBuildWithExtraProperties() { + Table table = + mockTableWithExtraProperties( + "hello", + ImmutableMap.of( + "properties.ssl.truststore.location", + "/path/to/kafka.client.truststore.jks", + "properties.security.protocol", + "SASL_SSL")); + BeamSqlTable sqlTable = provider.buildBeamSqlTable(table); + + assertNotNull(sqlTable); + assertTrue(sqlTable instanceof BeamKafkaCSVTable); + + BeamKafkaCSVTable kafkaTable = (BeamKafkaCSVTable) sqlTable; + assertEquals(LOCATION_BROKER, kafkaTable.getBootstrapServers()); + assertEquals( + ImmutableMap.of( + "properties.ssl.truststore.location", + "/path/to/kafka.client.truststore.jks", + "properties.security.protocol", + "SASL_SSL"), + kafkaTable.getConfigUpdates()); + } + @Test public void testBuildBeamSqlAvroTable() { Table table = mockTable("hello", "avro"); @@ -157,23 +185,28 @@ public void testGetTableType() { } private static Table mockTable(String name) { - return mockTable(name, false, null, null, null, null, null, null); + return mockTable(name, false, null, null, null, null, null, null, null); } private static Table mockTableWithExtraServers(String name, List extraBootstrapServers) { - return mockTable(name, false, extraBootstrapServers, null, null, null, null, null); + return mockTable(name, false, extraBootstrapServers, null, null, null, null, null, null); } private static Table mockTableWithExtraTopics(String name, List extraTopics) { - return mockTable(name, false, null, extraTopics, null, null, null, null); + return mockTable(name, false, null, extraTopics, null, null, null, null, null); + } + + private static Table mockTableWithExtraProperties( + String name, Map extraProperties) { + return mockTable(name, false, null, null, extraProperties, null, null, null, null); } private static Table mockTable(String name, String payloadFormat) { - return mockTable(name, false, null, null, payloadFormat, null, null, null); + return mockTable(name, false, null, null, null, payloadFormat, null, null, null); } private static Table mockProtoTable(String name, Class protoClass) { - return mockTable(name, false, null, null, "proto", protoClass, null, null); + return mockTable(name, false, null, null, null, "proto", protoClass, null, null); } private static Table mockThriftTable( @@ -181,11 +214,11 @@ private static Table mockThriftTable( Class> thriftClass, Class thriftProtocolFactoryClass) { return mockTable( - name, false, null, null, "thrift", null, thriftClass, thriftProtocolFactoryClass); + name, false, null, null, null, "thrift", null, thriftClass, thriftProtocolFactoryClass); } private static Table mockNestedBytesTable(String name) { - return mockTable(name, true, null, null, null, null, null, null); + return mockTable(name, true, null, null, null, null, null, null, null); } private static Table mockNestedThriftTable( @@ -193,7 +226,7 @@ private static Table mockNestedThriftTable( Class> thriftClass, Class thriftProtocolFactoryClass) { return mockTable( - name, true, null, null, "thrift", null, thriftClass, thriftProtocolFactoryClass); + name, true, null, null, null, "thrift", null, thriftClass, thriftProtocolFactoryClass); } private static Table mockTable( @@ -201,6 +234,7 @@ private static Table mockTable( boolean isNested, @Nullable List extraBootstrapServers, @Nullable List extraTopics, + @Nullable Map extraProperties, @Nullable String payloadFormat, @Nullable Class protoClass, @Nullable Class> thriftClass, @@ -222,6 +256,12 @@ private static Table mockTable( properties.put("topics", topics); } + if (extraProperties != null) { + for (Map.Entry property : extraProperties.entrySet()) { + properties.put(property.getKey(), property.getValue()); + } + } + if (payloadFormat != null) { properties.put("format", payloadFormat); } From 4241b312ace8ca818b1274257d4971f8e9589524 Mon Sep 17 00:00:00 2001 From: Talat Uyarer Date: Tue, 18 Mar 2025 11:15:21 -0700 Subject: [PATCH 02/48] Need to remove properies prefix from properties to apply on Kafka --- .../sql/meta/provider/kafka/KafkaTableProvider.java | 2 +- .../sql/meta/provider/kafka/KafkaTableProviderTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java index 6dd6d13428de..fb2fab4c8562 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java @@ -148,7 +148,7 @@ public BeamSqlTable buildBeamSqlTable(Table table) { while (properties.fields().hasNext()) { Entry field = properties.fields().next(); if (field.getKey().startsWith("properties.")) { - configUpdates.put(field.getKey(), field.getValue().textValue()); + configUpdates.put(field.getKey().replace("properties.", ""), field.getValue().textValue()); } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java index a5a6e7c55a65..213d557cd97f 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderTest.java @@ -105,9 +105,9 @@ public void testBuildWithExtraProperties() { assertEquals(LOCATION_BROKER, kafkaTable.getBootstrapServers()); assertEquals( ImmutableMap.of( - "properties.ssl.truststore.location", + "ssl.truststore.location", "/path/to/kafka.client.truststore.jks", - "properties.security.protocol", + "security.protocol", "SASL_SSL"), kafkaTable.getConfigUpdates()); } From b2ef3b067464b6e7b56a15d72fd343203090e610 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 18 Mar 2025 14:30:42 -0400 Subject: [PATCH 03/48] Expose add-modules JVM option in SDKHarnessOptions (#34289) * Expose add-modules JVM option in SDKHarnessOptions * add changes.md * Update CHANGES.md Co-authored-by: tvalentyn * Fix whitespace * fix whitespace --------- Co-authored-by: tvalentyn --- CHANGES.md | 3 ++- sdks/java/container/boot.go | 8 +++++++- .../beam/sdk/options/SdkHarnessOptions.java | 19 ++++++++++++++++--- 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index a40bc99651f4..261cab20719f 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -69,8 +69,9 @@ ## New Features / Improvements -* Support custom coders in Reshuffle ([#29908](https://github.com/apache/beam/issues/29908), [#33356](https://github.com/apache/beam/issues/33356)). +* [Python] Support custom coders in Reshuffle ([#29908](https://github.com/apache/beam/issues/29908), [#33356](https://github.com/apache/beam/issues/33356)). * [Java] Upgrade SLF4J to 2.0.16. Update default Spark version to 3.5.0. ([#33574](https://github.com/apache/beam/pull/33574)) +* [Java] Support for `--add-modules` JVM option is added through a new pipeline option `JdkAddRootModules`. This allows extending the module graph with optional modules such as SDK incubator modules. Sample usage: ` --jdkAddRootModules=jdk.incubator.vector` ([#30281](https://github.com/apache/beam/issues/30281)). * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Managed API for [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/managed/Managed.html) and [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.managed.html#module-apache_beam.transforms.managed) supports [key I/O connectors](https://beam.apache.org/documentation/io/connectors/) Iceberg, Kafka, and BigQuery. diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go index 19b7eee0bba4..7169c2628dea 100644 --- a/sdks/java/container/boot.go +++ b/sdks/java/container/boot.go @@ -252,13 +252,19 @@ func main() { sort.Strings(properties) args = append(args, properties...) - // Open modules specified in pipeline options if pipelineOptions, ok := info.GetPipelineOptions().GetFields()["options"]; ok { + // Open modules specified in pipeline options if modules, ok := pipelineOptions.GetStructValue().GetFields()["jdkAddOpenModules"]; ok { for _, module := range modules.GetListValue().GetValues() { args = append(args, "--add-opens="+module.GetStringValue()) } } + // Add modules specified in pipeline options + if modules, ok := pipelineOptions.GetStructValue().GetFields()["jdkAddRootModules"]; ok { + for _, module := range modules.GetListValue().GetValues() { + args = append(args, "--add-modules="+module.GetStringValue()) + } + } } // Automatically open modules for Java 11+ openModuleAgentJar := "/opt/apache/beam/jars/open-module-agent.jar" diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java index 2981046a0a41..f56196996bcc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java @@ -349,13 +349,15 @@ public static SdkHarnessLogLevelOverrides from(Map values) { } /** - * Open modules needed for reflection that access JDK internals with Java 9+ + * Open modules needed for reflection that access JDK internals with Java 9+. * *

With JDK 16+, JDK internals are strongly * encapsulated and can result in an InaccessibleObjectException being thrown if a tool or * library uses reflection that access JDK internals. If you see these errors in your worker logs, - * you can pass in modules to open using the format module/package=target-module(,target-module)* - * to allow access to the library. E.g. java.base/java.lang=jamm + * you can pass in modules to open using the format {@code + * module/package=target-module[,module2/package2=another-target-module]} to allow access to the + * library. E.g. {@code --jdkAddOpenModules=java.base/java.lang=jamm}. This will set {@code + * --add-opens} JVM flag in SDK Harness invocation. * *

You may see warnings that jamm, a library used to more accurately size objects, is unable to * make a private field accessible. To resolve the warning, open the specified module/package to @@ -366,6 +368,17 @@ public static SdkHarnessLogLevelOverrides from(Map values) { void setJdkAddOpenModules(List options); + /** + * Add modules to the default root set with Java 11+. + * + *

Set {@code --add-modules} JVM flag in SDK Harness invocation. E.g. {@code + * --jdkAddModules=module1,module2}. + */ + @Description("Add modules to the default root set with Java 11+.") + List getJdkAddRootModules(); + + void setJdkAddRootModules(List options); + /** * Configure log manager's default log level and log level overrides from the sdk harness options, * and return the list of configured loggers. From 10031946cfc6d95bb17deb28cf5e385c1448206a Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Tue, 18 Mar 2025 14:37:50 -0400 Subject: [PATCH 04/48] [AnomalyDetection] Refactor and improve Specifiable (#34310) * Refactor and improve Specifiable. - Better typehinting in specifiable with overloads - Move non-method attributes and staticmethod outside of Specifiable Protocol - Make spec_type a function which returns a class-specific spec_type variable - Add tests for some uncommon use cases. * Fix static function names and use pass for empty functions per review. * Add a warning for unsupported types in spec. --- sdks/python/apache_beam/ml/anomaly/base.py | 4 +- .../apache_beam/ml/anomaly/specifiable.py | 175 ++++++++++++------ .../ml/anomaly/specifiable_test.py | 54 +++++- 3 files changed, 166 insertions(+), 67 deletions(-) diff --git a/sdks/python/apache_beam/ml/anomaly/base.py b/sdks/python/apache_beam/ml/anomaly/base.py index e3c6252474bd..8251245f1cac 100644 --- a/sdks/python/apache_beam/ml/anomaly/base.py +++ b/sdks/python/apache_beam/ml/anomaly/base.py @@ -154,7 +154,7 @@ def __init__( threshold_criterion: Optional[ThresholdFn] = None, **kwargs): self._model_id = model_id if model_id is not None else getattr( - self, 'spec_type', 'unknown') + self, 'spec_type', lambda: "unknown")() self._features = features self._target = target self._threshold_criterion = threshold_criterion @@ -200,7 +200,7 @@ def __init__( aggregation_strategy: Optional[AggregationFn] = None, **kwargs): if "model_id" not in kwargs or kwargs["model_id"] is None: - kwargs["model_id"] = getattr(self, 'spec_type', 'custom') + kwargs["model_id"] = getattr(self, 'spec_type', lambda: 'custom')() super().__init__(**kwargs) diff --git a/sdks/python/apache_beam/ml/anomaly/specifiable.py b/sdks/python/apache_beam/ml/anomaly/specifiable.py index 2eeb1d0de76d..3a2baf434f9b 100644 --- a/sdks/python/apache_beam/ml/anomaly/specifiable.py +++ b/sdks/python/apache_beam/ml/anomaly/specifiable.py @@ -27,7 +27,7 @@ import logging import os from typing import Any -from typing import ClassVar +from typing import Callable from typing import Dict from typing import List from typing import Optional @@ -35,6 +35,7 @@ from typing import Type from typing import TypeVar from typing import Union +from typing import overload from typing import runtime_checkable from typing_extensions import Self @@ -59,7 +60,8 @@ #: `spec_type` when applying the `specifiable` decorator to an existing class. _KNOWN_SPECIFIABLE = collections.defaultdict(dict) -SpecT = TypeVar('SpecT', bound='Specifiable') +T = TypeVar('T', bound=type) +BUILTIN_TYPES_IN_SPEC = (int, float, complex, str, bytes, bytearray) def _class_to_subspace(cls: Type) -> str: @@ -104,33 +106,59 @@ class Spec(): config: Optional[Dict[str, Any]] = dataclasses.field(default_factory=dict) -@runtime_checkable -class Specifiable(Protocol): - """Protocol that a specifiable class needs to implement.""" - #: The value of the `type` field in the object's spec for this class. - spec_type: ClassVar[str] - #: The raw keyword arguments passed to `__init__` method during object - #: initialization. - init_kwargs: dict[str, Any] +def _specifiable_from_spec_helper(v, _run_init): + if isinstance(v, Spec): + return Specifiable.from_spec(v, _run_init) + + if isinstance(v, List): + return [_specifiable_from_spec_helper(e, _run_init) for e in v] + + # TODO: support spec treatment for more types + if not isinstance(v, BUILTIN_TYPES_IN_SPEC): + logging.warning( + "Type %s is not a recognized supported type for the" + "specification. It will be included without conversion.", + str(type(v))) + return v - # a boolean to tell whether the original `__init__` method is called - _initialized: bool - # a boolean used by new_getattr to tell whether it is in the `__init__` method - # call - _in_init: bool - @staticmethod - def _from_spec_helper(v, _run_init): - if isinstance(v, Spec): - return Specifiable.from_spec(v, _run_init) +def _specifiable_to_spec_helper(v): + if isinstance(v, Specifiable): + return v.to_spec() - if isinstance(v, List): - return [Specifiable._from_spec_helper(e, _run_init) for e in v] + if isinstance(v, List): + return [_specifiable_to_spec_helper(e) for e in v] - return v + if inspect.isfunction(v): + if not hasattr(v, "spec_type"): + _register(v, inject_spec_type=False) + return Spec(type=_get_default_spec_type(v), config=None) + if inspect.isclass(v): + if not hasattr(v, "spec_type"): + _register(v, inject_spec_type=False) + return Spec(type=_get_default_spec_type(v), config=None) + + # TODO: support spec treatment for more types + if not isinstance(v, BUILTIN_TYPES_IN_SPEC): + logging.warning( + "Type %s is not a recognized supported type for the" + "specification. It will be included without conversion.", + str(type(v))) + return v + + +@runtime_checkable +class Specifiable(Protocol): + """Protocol that a specifiable class needs to implement.""" @classmethod - def from_spec(cls, spec: Spec, _run_init: bool = True) -> Union[Self, type]: + def spec_type(cls) -> str: + pass + + @classmethod + def from_spec(cls, + spec: Spec, + _run_init: bool = True) -> Union[Self, type[Self]]: """Generate a `Specifiable` subclass object based on a spec. Args: @@ -155,7 +183,7 @@ def from_spec(cls, spec: Spec, _run_init: bool = True) -> Union[Self, type]: return subclass kwargs = { - k: Specifiable._from_spec_helper(v, _run_init) + k: _specifiable_from_spec_helper(v, _run_init) for k, v in spec.config.items() } @@ -164,26 +192,6 @@ def from_spec(cls, spec: Spec, _run_init: bool = True) -> Union[Self, type]: kwargs["_run_init"] = True return subclass(**kwargs) - @staticmethod - def _to_spec_helper(v): - if isinstance(v, Specifiable): - return v.to_spec() - - if isinstance(v, List): - return [Specifiable._to_spec_helper(e) for e in v] - - if inspect.isfunction(v): - if not hasattr(v, "spec_type"): - _register(v, inject_spec_type=False) - return Spec(type=_get_default_spec_type(v), config=None) - - if inspect.isclass(v): - if not hasattr(v, "spec_type"): - _register(v, inject_spec_type=False) - return Spec(type=_get_default_spec_type(v), config=None) - - return v - def to_spec(self) -> Spec: """Generate a spec from a `Specifiable` subclass object. @@ -195,14 +203,22 @@ def to_spec(self) -> Spec: f"'{type(self).__name__}' not registered as Specifiable. " f"Decorate ({type(self).__name__}) with @specifiable") - args = {k: self._to_spec_helper(v) for k, v in self.init_kwargs.items()} + args = { + k: _specifiable_to_spec_helper(v) + for k, v in self.init_kwargs.items() + } - return Spec(type=self.__class__.spec_type, config=args) + return Spec(type=self.spec_type(), config=args) def run_original_init(self) -> None: """Invoke the original __init__ method with original keyword arguments""" pass + @classmethod + def unspecifiable(cls) -> None: + """Resume the class structure prior to specifiable""" + pass + def _get_default_spec_type(cls): spec_type = cls.__name__ @@ -216,7 +232,7 @@ def _get_default_spec_type(cls): # Register a `Specifiable` subclass in `KNOWN_SPECIFIABLE` -def _register(cls, spec_type=None, inject_spec_type=True) -> None: +def _register(cls: type, spec_type=None, inject_spec_type=True) -> None: assert spec_type is None or inject_spec_type, \ "need to inject spec_type to class if spec_type is not None" if spec_type is None: @@ -237,7 +253,8 @@ def _register(cls, spec_type=None, inject_spec_type=True) -> None: _KNOWN_SPECIFIABLE[subspace][spec_type] = cls if inject_spec_type: - cls.spec_type = spec_type + setattr(cls, cls.__name__ + '__spec_type', spec_type) + # cls.__spec_type = spec_type # Keep a copy of arguments that are used to call the `__init__` method when the @@ -250,13 +267,35 @@ def _get_init_kwargs(inst, init_method, *args, **kwargs): return params +@overload +def specifiable( + my_cls: None = None, + /, + *, + spec_type: Optional[str] = None, + on_demand_init: bool = True, + just_in_time_init: bool = True) -> Callable[[T], T]: + pass + + +@overload def specifiable( - my_cls=None, + my_cls: T, /, *, - spec_type=None, - on_demand_init=True, - just_in_time_init=True): + spec_type: Optional[str] = None, + on_demand_init: bool = True, + just_in_time_init: bool = True) -> T: + pass + + +def specifiable( + my_cls: Optional[T] = None, + /, + *, + spec_type: Optional[str] = None, + on_demand_init: bool = True, + just_in_time_init: bool = True) -> Union[T, Callable[[T], T]]: """A decorator that turns a class into a `Specifiable` subclass by implementing the `Specifiable` protocol. @@ -285,8 +324,8 @@ class Bar(): original `__init__` method will be called when the first time an attribute is accessed. """ - def _wrapper(cls): - def new_init(self: Specifiable, *args, **kwargs): + def _wrapper(cls: T) -> T: + def new_init(self, *args, **kwargs): self._initialized = False self._in_init = False @@ -358,20 +397,40 @@ def new_getattr(self, name): name) return self.__getattribute__(name) + def spec_type_func(cls): + return getattr(cls, spec_type_attr_name) + + def unspecifiable(cls): + delattr(cls, spec_type_attr_name) + cls.__init__ = original_init + if just_in_time_init: + delattr(cls, '__getattr__') + delattr(cls, 'spec_type') + delattr(cls, 'run_original_init') + delattr(cls, 'to_spec') + delattr(cls, 'from_spec') + delattr(cls, 'unspecifiable') + + spec_type_attr_name = cls.__name__ + "__spec_type" + + # the class is registered + if hasattr(cls, spec_type_attr_name): + return cls + # start of the function body of _wrapper _register(cls, spec_type) class_name = cls.__name__ - original_init = cls.__init__ - cls.__init__ = new_init + original_init = cls.__init__ # type: ignore[misc] + cls.__init__ = new_init # type: ignore[misc] if just_in_time_init: cls.__getattr__ = new_getattr + cls.spec_type = classmethod(spec_type_func) cls.run_original_init = run_original_init cls.to_spec = Specifiable.to_spec - cls._to_spec_helper = staticmethod(Specifiable._to_spec_helper) cls.from_spec = Specifiable.from_spec - cls._from_spec_helper = staticmethod(Specifiable._from_spec_helper) + cls.unspecifiable = classmethod(unspecifiable) return cls # end of the function body of _wrapper diff --git a/sdks/python/apache_beam/ml/anomaly/specifiable_test.py b/sdks/python/apache_beam/ml/anomaly/specifiable_test.py index 4c1a7bdaf32a..4492cbbe4104 100644 --- a/sdks/python/apache_beam/ml/anomaly/specifiable_test.py +++ b/sdks/python/apache_beam/ml/anomaly/specifiable_test.py @@ -52,7 +52,7 @@ class B(): # apply the decorator function to an existing class A = specifiable(A) - self.assertEqual(A.spec_type, "A") + self.assertEqual(A.spec_type(), "A") self.assertTrue(isinstance(A(), Specifiable)) self.assertIn("A", _KNOWN_SPECIFIABLE[_FALLBACK_SUBSPACE]) self.assertEqual(_KNOWN_SPECIFIABLE[_FALLBACK_SUBSPACE]["A"], A) @@ -63,13 +63,10 @@ class B(): # Raise an error when re-registering spec_type with a different class self.assertRaises(ValueError, specifiable(spec_type='A'), B) - # apply the decorator function to an existing class with a different - # spec_type + # Applying the decorator function to an existing class with a different + # spec_type will have no effect. A = specifiable(spec_type="A_DUP")(A) - self.assertEqual(A.spec_type, "A_DUP") - self.assertTrue(isinstance(A(), Specifiable)) - self.assertIn("A_DUP", _KNOWN_SPECIFIABLE[_FALLBACK_SUBSPACE]) - self.assertEqual(_KNOWN_SPECIFIABLE[_FALLBACK_SUBSPACE]["A_DUP"], A) + self.assertEqual(A.spec_type(), "A") def test_decorator_in_syntactic_sugar_form(self): # call decorator without parameters @@ -585,6 +582,49 @@ def apply(self, x, y): self.assertEqual(w_2.run_func_in_class(5, 3), 150) +class TestUncommonUsages(unittest.TestCase): + def test_double_specifiable(self): + @specifiable + @specifiable + class ZZ(): + def __init__(self, a): + self.a = a + + assert issubclass(ZZ, Specifiable) + c = ZZ("b") + c.run_original_init() + self.assertEqual(c.a, "b") + + def test_unspecifiable(self): + class YY(): + def __init__(self, x): + self.x = x + assert False + + YY = specifiable(YY) + assert issubclass(YY, Specifiable) + y = YY(1) + # __init__ is called (with assertion error raised) when attribute is first + # accessed + self.assertRaises(AssertionError, lambda: y.x) + + # unspecifiable YY + YY.unspecifiable() + # __init__ is called immediately + self.assertRaises(AssertionError, YY, 1) + self.assertFalse(hasattr(YY, 'run_original_init')) + self.assertFalse(hasattr(YY, 'spec_type')) + self.assertFalse(hasattr(YY, 'to_spec')) + self.assertFalse(hasattr(YY, 'from_spec')) + self.assertFalse(hasattr(YY, 'unspecifiable')) + + # make YY specifiable again + YY = specifiable(YY) + assert issubclass(YY, Specifiable) + y = YY(1) + self.assertRaises(AssertionError, lambda: y.x) + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() From 33fd2bbe855aa6849444bb35b47ac5106eea61de Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Mar 2025 15:02:40 -0400 Subject: [PATCH 05/48] Bump golang.org/x/oauth2 from 0.26.0 to 0.28.0 in /sdks (#34190) Bumps [golang.org/x/oauth2](https://github.com/golang/oauth2) from 0.26.0 to 0.28.0. - [Commits](https://github.com/golang/oauth2/compare/v0.26.0...v0.28.0) --- updated-dependencies: - dependency-name: golang.org/x/oauth2 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 a2f94d918fcf..770fe13463f9 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -56,7 +56,7 @@ require ( github.com/xitongsys/parquet-go-source v0.0.0-20241021075129-b732d2ac9c9b go.mongodb.org/mongo-driver v1.17.3 golang.org/x/net v0.37.0 - golang.org/x/oauth2 v0.26.0 + golang.org/x/oauth2 v0.28.0 golang.org/x/sync v0.12.0 golang.org/x/sys v0.31.0 golang.org/x/text v0.23.0 diff --git a/sdks/go.sum b/sdks/go.sum index 5ea18d8a1df6..1dddf2630c38 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1700,8 +1700,8 @@ golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I= golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= -golang.org/x/oauth2 v0.26.0 h1:afQXWNNaeC4nvZ0Ed9XvCCzXM6UHJG7iCg0W4fPqSBE= -golang.org/x/oauth2 v0.26.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.28.0 h1:CrgCKl8PPAVtLnU3c+EDw6x11699EWlsDeWNWKdIOkc= +golang.org/x/oauth2 v0.28.0/go.mod h1:onh5ek6nERTohokkhCD/y2cV4Do3fxFHFuAejCkRWT8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= From c3c10d2d3f21a9cedc448ff5c680d69d112ba469 Mon Sep 17 00:00:00 2001 From: liferoad Date: Tue, 18 Mar 2025 15:07:21 -0400 Subject: [PATCH 06/48] Support directory separator for Python local filesystem (#34318) * Support directory separator for Python local filesystem * fixed the tests --- sdks/python/apache_beam/io/textio.py | 3 +++ sdks/python/apache_beam/io/textio_test.py | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 0d7803bcabb1..cd693c584db3 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -20,6 +20,7 @@ # pytype: skip-file import logging +import os from functools import partial from typing import TYPE_CHECKING from typing import Any @@ -781,6 +782,8 @@ def __init__( """ super().__init__(**kwargs) + if file_pattern and not os.path.dirname(file_pattern): + file_pattern = os.path.join('.', file_pattern) self._source = self._source_class( file_pattern, min_bundle_size, diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index d1bfdf6bfd35..580db01253f0 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -23,6 +23,7 @@ import gzip import logging import os +import platform import shutil import tempfile import unittest @@ -197,6 +198,23 @@ def test_read_single_file(self): assert len(expected_data) == TextSourceTest.DEFAULT_NUM_RECORDS self._run_read_test(file_name, expected_data) + @unittest.skipIf(platform.system() == 'Windows', 'Skipping on Windows') + def test_read_from_text_file_pattern_with_dot_slash(self): + cwd = os.getcwd() + expected = ['abc', 'de'] + with TempDir() as temp_dir: + temp_dir.create_temp_file(suffix='.txt', lines=[b'a', b'b', b'c']) + temp_dir.create_temp_file(suffix='.txt', lines=[b'd', b'e']) + + os.chdir(temp_dir.get_path()) + with TestPipeline() as p: + dot_slash = p | 'ReadDotSlash' >> ReadFromText('./*.txt') + no_dot_slash = p | 'ReadNoSlash' >> ReadFromText('*.txt') + + assert_that(dot_slash, equal_to(expected)) + assert_that(no_dot_slash, equal_to(expected)) + os.chdir(cwd) + def test_read_single_file_smaller_than_default_buffer(self): file_name, expected_data = write_data(TextSourceTest.DEFAULT_NUM_RECORDS) self._run_read_test( From 10c22cffbb957babca101a095736abb32d3a65de Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Mar 2025 15:33:58 -0400 Subject: [PATCH 07/48] Bump github.com/nats-io/nats.go from 1.39.0 to 1.39.1 in /sdks (#34029) Bumps [github.com/nats-io/nats.go](https://github.com/nats-io/nats.go) from 1.39.0 to 1.39.1. - [Release notes](https://github.com/nats-io/nats.go/releases) - [Commits](https://github.com/nats-io/nats.go/compare/v1.39.0...v1.39.1) --- updated-dependencies: - dependency-name: github.com/nats-io/nats.go 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 | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 770fe13463f9..ce2c9034100c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -47,7 +47,7 @@ require ( github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.1 github.com/nats-io/nats-server/v2 v2.10.25 - github.com/nats-io/nats.go v1.39.0 + github.com/nats-io/nats.go v1.39.1 github.com/proullon/ramsql v0.1.4 github.com/spf13/cobra v1.9.1 github.com/testcontainers/testcontainers-go v0.35.0 diff --git a/sdks/go.sum b/sdks/go.sum index 1dddf2630c38..b36a229e24ee 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1306,8 +1306,8 @@ github.com/nats-io/jwt/v2 v2.7.3 h1:6bNPK+FXgBeAqdj4cYQ0F8ViHRbi7woQLq4W29nUAzE= github.com/nats-io/jwt/v2 v2.7.3/go.mod h1:GvkcbHhKquj3pkioy5put1wvPxs78UlZ7D/pY+BgZk4= github.com/nats-io/nats-server/v2 v2.10.25 h1:J0GWLDDXo5HId7ti/lTmBfs+lzhmu8RPkoKl0eSCqwc= github.com/nats-io/nats-server/v2 v2.10.25/go.mod h1:/YYYQO7cuoOBt+A7/8cVjuhWTaTUEAlZbJT+3sMAfFU= -github.com/nats-io/nats.go v1.39.0 h1:2/yg2JQjiYYKLwDuBzV0FbB2sIV+eFNkEevlRi4n9lI= -github.com/nats-io/nats.go v1.39.0/go.mod h1:MgRb8oOdigA6cYpEPhXJuRVH6UE/V4jblJ2jQ27IXYM= +github.com/nats-io/nats.go v1.39.1 h1:oTkfKBmz7W047vRxV762M67ZdXeOtUgvbBaNoQ+3PPk= +github.com/nats-io/nats.go v1.39.1/go.mod h1:MgRb8oOdigA6cYpEPhXJuRVH6UE/V4jblJ2jQ27IXYM= github.com/nats-io/nkeys v0.4.9 h1:qe9Faq2Gxwi6RZnZMXfmGMZkg3afLLOtrU+gDZJ35b0= github.com/nats-io/nkeys v0.4.9/go.mod h1:jcMqs+FLG+W5YO36OX6wFIFcmpdAns+w1Wm6D3I/evE= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= From 09e9993b500dda5fb8e40ad24d1a1480781d68f0 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 25 Feb 2025 08:35:46 -0800 Subject: [PATCH 08/48] Make yaml provider an abstract class. --- sdks/python/apache_beam/yaml/yaml_provider.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 97a84c068a4a..36b7482b1e18 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -19,6 +19,7 @@ for where to find and how to invoke services that vend implementations of various PTransforms.""" +import abc import collections import functools import hashlib @@ -78,15 +79,18 @@ def __bool__(self): return False -class Provider: +class Provider(abc.ABC): """Maps transform types names and args to concrete PTransform instances.""" + @abc.abstractmethod def available(self) -> Union[bool, NotAvailableWithReason]: """Returns whether this provider is available to use in this environment.""" raise NotImplementedError(type(self)) + @abc.abstractmethod def cache_artifacts(self) -> Optional[Iterable[str]]: raise NotImplementedError(type(self)) + @abc.abstractmethod def provided_transforms(self) -> Iterable[str]: """Returns a list of transform type names this provider can handle.""" raise NotImplementedError(type(self)) @@ -107,6 +111,7 @@ def requires_inputs(self, typ: str, args: Mapping[str, Any]) -> bool: """ return not typ.startswith('Read') + @abc.abstractmethod def create_transform( self, typ: str, From 1c0a36230fbbd4744456a9344cf17eedfc96d7f7 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 21 Feb 2025 16:26:18 -0800 Subject: [PATCH 09/48] Add support for python package dependencies on UDFs. --- .../elementwise/map_to_fields_with_deps.yaml | 51 +++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_mapping.py | 5 +- sdks/python/apache_beam/yaml/yaml_provider.py | 37 ++++++++++++++ .../python/apache_beam/yaml/yaml_transform.py | 10 ++++ 4 files changed, 102 insertions(+), 1 deletion(-) create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_deps.yaml diff --git a/sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_deps.yaml b/sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_deps.yaml new file mode 100644 index 000000000000..a45f6ceb98c0 --- /dev/null +++ b/sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_deps.yaml @@ -0,0 +1,51 @@ +# coding=utf-8 +# +# 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. +# + +pipeline: + type: chain + transforms: + - type: Create + config: + elements: + - {sdk: MapReduce, year: 2004} + - {sdk: MillWheel, year: 2008} + - {sdk: Flume, year: 2010} + - {sdk: Dataflow, year: 2014} + - {sdk: Apache Beam, year: 2016} + - type: MapToFields + name: ToRoman + config: + language: python + fields: + tool_name: sdk + year: + callable: | + import roman + + def convert(row): + return roman.toRoman(row.year) + dependencies: + - 'roman>=4.2' + - type: LogForTesting + +# Expected: +# Row(tool_name='MapReduce', year='MMIV') +# Row(tool_name='MillWheel', year='MMVIII') +# Row(tool_name='Flume', year='MMX') +# Row(tool_name='Dataflow', year='MMXIV') +# Row(tool_name='Apache Beam', year='MMXVI') diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py index 7ffee5c2039b..4f7133838794 100644 --- a/sdks/python/apache_beam/yaml/yaml_mapping.py +++ b/sdks/python/apache_beam/yaml/yaml_mapping.py @@ -678,7 +678,8 @@ def _PyJsMapToFields( fields: Mapping[str, Union[str, Mapping[str, str]]], append: Optional[bool] = False, drop: Optional[Iterable[str]] = None, - language: Optional[str] = None): + language: Optional[str] = None, + dependencies: Optional[Iterable[str]] = None): """Creates records with new fields defined in terms of the input fields. See more complete documentation on @@ -694,6 +695,8 @@ def _PyJsMapToFields( original record that should not be kept language: The language used to define (and execute) the expressions and/or callables in `fields`. Defaults to generic. + dependencies: An optional list of extra dependencies that are needed for + these UDFs. The interpretation of these strings is language-dependent. error_handling: Whether and where to output records that throw errors when the above expressions are evaluated. """ # pylint: disable=line-too-long diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 36b7482b1e18..04f2a0f0dafb 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -150,6 +150,13 @@ def _affinity(self, other: "Provider"): else: return 0 + @functools.cache + def with_extra_dependencies(self, dependencies: Iterable[str]): + return self._with_extra_dependencies(dependencies) + + def _with_extra_dependencies(self, dependencies: Iterable[str]): + raise ValueError('This provider does not support additional dependencies.') + def as_provider(name, provider_or_constructor): if isinstance(provider_or_constructor, Provider): @@ -611,6 +618,18 @@ def requires_inputs(self, typ, args): else: return super().requires_inputs(typ, args) + def _with_extra_dependencies(self, dependencies): + external_provider = ExternalPythonProvider( # + { + typ: 'apache_beam.yaml.yaml_provider.standard_inline_providers.' + + typ.replace('-', '_') + for typ in self._transform_factories.keys() + }, + '__inline__', + dependencies) + external_provider.to_json = self.to_json + return external_provider + class MetaInlineProvider(InlineProvider): def create_transform(self, type, args, yaml_create_transform): @@ -1381,6 +1400,7 @@ def merge_providers(*provider_sets) -> Mapping[str, Iterable[Provider]]: return result +@functools.cache def standard_providers(): from apache_beam.yaml.yaml_combine import create_combine_providers from apache_beam.yaml.yaml_mapping import create_mapping_providers @@ -1407,3 +1427,20 @@ def _file_digest(fileobj, digest): hasher.update(data) data = fileobj.read(1 << 20) return hasher + + +class _InlineProviderNamespace: + """Gives fully qualified names to inline providers from standard_providers(). + + This is needed to upgrade InlineProvider to ExternalPythonProvider. + """ + def __getattr__(self, name): + typ = name.replace('_', '-') + for provider in standard_providers()[typ]: + if isinstance(provider, InlineProvider): + return provider._transform_factories[typ] + else: + raise ValueError(f"No inline provider found for {name}") + + +standard_inline_providers = _InlineProviderNamespace() diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 66ce87c9bfe1..5929d07402f3 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -368,6 +368,9 @@ def expand(pcolls): if pcoll in providers_by_input ] provider = self.best_provider(spec, input_providers) + extra_dependencies = extract_extra_dependencies(spec) + if extra_dependencies: + provider = provider.with_extra_dependencies(frozenset(extra_dependencies)) config = SafeLineLoader.strip_metadata(spec.get('config', {})) if not isinstance(config, dict): @@ -708,6 +711,13 @@ def extract_name(spec): return '' +def extract_extra_dependencies(spec): + deps = spec.get('config', {}).get('dependencies', []) + if not isinstance(deps, list): + raise TypeErrorError(f'Dependencies must be a list of strings, got {deps}') + return deps + + def push_windowing_to_roots(spec): scope = LightweightScope(spec['transforms']) consumed_outputs_by_transform = collections.defaultdict(set) From 7126636c1327087e5ff9ebc05a8d95c335b3fcfc Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 25 Feb 2025 09:23:48 -0800 Subject: [PATCH 10/48] Add dependency support for delegating providers. --- sdks/python/apache_beam/yaml/yaml_provider.py | 28 +++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 04f2a0f0dafb..895f953373c5 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -152,10 +152,15 @@ def _affinity(self, other: "Provider"): @functools.cache def with_extra_dependencies(self, dependencies: Iterable[str]): - return self._with_extra_dependencies(dependencies) + result = self._with_extra_dependencies(dependencies) + if not hasattr(result, 'to_json'): + result.to_json = lambda: {'type': type(result).__name__} + return result def _with_extra_dependencies(self, dependencies: Iterable[str]): - raise ValueError('This provider does not support additional dependencies.') + raise ValueError( + 'This provider of type %s does not support additional dependencies.' % + type(self).__name__) def as_provider(name, provider_or_constructor): @@ -404,6 +409,8 @@ def is_path_or_urn(package): if is_path_or_urn(package) else package for package in packages ])) + self._packages = packages + def available(self): return True # If we're running this script, we have Python installed. @@ -430,6 +437,10 @@ def _affinity(self, other: "Provider"): else: return super()._affinity(other) + def _with_extra_dependencies(self, dependencies: Iterable[str]): + return ExternalPythonProvider( + self._urns, None, set(self._packages) + set(dependencies)) + @ExternalProvider.register_provider_type('yaml') class YamlProvider(Provider): @@ -1041,6 +1052,11 @@ def create_transform( yaml_create_transform: Any) -> beam.PTransform: return self._transforms[typ](self._underlying_provider, **config) + def _with_extra_dependencies(self, dependencies: Iterable[str]): + return TranslatingProvider( + self._transforms, + self._underlying_provider._with_extra_dependencies(dependencies)) + def create_java_builtin_provider(): """Exposes built-in transforms from Java as well as Python to maximize @@ -1320,6 +1336,14 @@ def underlying_provider(self): def cache_artifacts(self): self._underlying_provider.cache_artifacts() + def _with_extra_dependencies(self, dependencies: Iterable[str]): + return RenamingProvider( + self._transforms, + None, + self._mappings, + self._underlying_provider._with_extra_dependencies(dependencies), + self._defaults) + def _as_list(func): @functools.wraps(func) From a47fa9b1ff3e9cb13d23eb283f94a06791660b1b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 25 Feb 2025 12:57:42 -0800 Subject: [PATCH 11/48] Allow dependency declarations for java. --- .../map_to_fields_with_java_deps.yaml | 57 +++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 16 +++++- .../python/apache_beam/yaml/yaml_transform.py | 8 ++- 3 files changed, 77 insertions(+), 4 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_java_deps.yaml diff --git a/sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_java_deps.yaml b/sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_java_deps.yaml new file mode 100644 index 000000000000..e32ac1e7b67c --- /dev/null +++ b/sdks/python/apache_beam/yaml/examples/transforms/elementwise/map_to_fields_with_java_deps.yaml @@ -0,0 +1,57 @@ +# coding=utf-8 +# +# 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. +# + +pipeline: + type: chain + transforms: + - type: Create + config: + elements: + - {sdk: MapReduce, year: 2004} + - {sdk: MillWheel, year: 2008} + - {sdk: Flume, year: 2010} + - {sdk: Dataflow, year: 2014} + - {sdk: Apache Beam, year: 2016} + - type: MapToFields + name: ToRoman + config: + language: java + fields: + tool_name: sdk + year: + callable: | + import org.apache.beam.sdk.values.Row; + import java.util.function.Function; + import com.github.chaosfirebolt.converter.RomanInteger; + + public class MyFunction implements Function { + public String apply(Row row) { + return RomanInteger.parse( + String.valueOf(row.getInt64("year"))).toString(); + } + } + dependencies: + - 'com.github.chaosfirebolt.converter:roman-numeral-converter:2.1.0' + - type: LogForTesting + +# Expected: +# Row(tool_name='MapReduce', year='MMIV') +# Row(tool_name='MillWheel', year='MMVIII') +# Row(tool_name='Flume', year='MMX') +# Row(tool_name='Dataflow', year='MMXIV') +# Row(tool_name='Apache Beam', year='MMXVI') diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 895f953373c5..239d1f124525 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -356,10 +356,13 @@ def cache_artifacts(self): class ExternalJavaProvider(ExternalProvider): - def __init__(self, urns, jar_provider): + def __init__(self, urns, jar_provider, classpath=None): super().__init__( - urns, lambda: external.JavaJarExpansionService(jar_provider())) + urns, + lambda: external.JavaJarExpansionService( + jar_provider(), classpath=classpath)) self._jar_provider = jar_provider + self._classpath = classpath def available(self): # pylint: disable=subprocess-run-check @@ -381,6 +384,15 @@ def try_decode(bs): def cache_artifacts(self): return [self._jar_provider()] + def _with_extra_dependencies(self, dependencies: Iterable[str]): + jars = sum(( + external.JavaJarExpansionService._expand_jars(dep) + for dep in dependencies), []) + return ExternalJavaProvider( + self._urns, + jar_provider=self._jar_provider, + classpath=(list(self._classpath or []) + list(jars))) + @ExternalProvider.register_provider_type('python') def python(urns, provider_base_path, packages=()): diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 5929d07402f3..75c5bd830940 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -368,7 +368,7 @@ def expand(pcolls): if pcoll in providers_by_input ] provider = self.best_provider(spec, input_providers) - extra_dependencies = extract_extra_dependencies(spec) + extra_dependencies, spec = extract_extra_dependencies(spec) if extra_dependencies: provider = provider.with_extra_dependencies(frozenset(extra_dependencies)) @@ -713,9 +713,13 @@ def extract_name(spec): def extract_extra_dependencies(spec): deps = spec.get('config', {}).get('dependencies', []) + if not deps: + return [], spec if not isinstance(deps, list): raise TypeErrorError(f'Dependencies must be a list of strings, got {deps}') - return deps + return deps, dict( + spec, + config={k: v for k, v in spec['config'].items() if k != 'dependencies'}) def push_windowing_to_roots(spec): From 56b3ff602fe6ede8174dc2f170cae140ce9d0349 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 3 Mar 2025 09:33:48 -0800 Subject: [PATCH 12/48] Make yapf, lint, and mypy happy, fix test. --- .../apache_beam/options/pipeline_options.py | 14 ++++++-- .../portability/expansion_service_main.py | 2 +- .../fn_api_runner/worker_handlers.py | 1 + .../apache_beam/utils/subprocess_server.py | 35 ++++++++++++++----- .../yaml/examples/testing/examples_test.py | 27 ++++++++++++++ .../apache_beam/yaml/yaml_enrichment_test.py | 3 ++ sdks/python/apache_beam/yaml/yaml_provider.py | 31 +++++++++------- .../python/apache_beam/yaml/yaml_transform.py | 2 +- sdks/python/setup.py | 4 ++- 9 files changed, 92 insertions(+), 27 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index fd6b9c5a0503..6de15b90790d 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -419,6 +419,14 @@ def get_all_options( _LOGGER.warning( 'Unknown pipeline options received: %s. Ignore if flags are ' 'used for internal purposes.' % (','.join(unknown_args))) + + seen = set() + + def add_new_arg(arg, **kwargs): + if arg not in seen: + parser.add_argument(arg, **kwargs) + seen.add(arg) + i = 0 while i < len(unknown_args): # End of argument parsing. @@ -432,12 +440,12 @@ def get_all_options( if i + 1 >= len(unknown_args) or unknown_args[i + 1].startswith('-'): split = unknown_args[i].split('=', 1) if len(split) == 1: - parser.add_argument(unknown_args[i], action='store_true') + add_new_arg(unknown_args[i], action='store_true') else: - parser.add_argument(split[0], type=str) + add_new_arg(split[0], type=str) i += 1 elif unknown_args[i].startswith('--'): - parser.add_argument(unknown_args[i], type=str) + add_new_arg(unknown_args[i], type=str) i += 2 else: # skip all binary flags used with '-' and not '--'. diff --git a/sdks/python/apache_beam/runners/portability/expansion_service_main.py b/sdks/python/apache_beam/runners/portability/expansion_service_main.py index 307f6bd54182..6b89cee6082e 100644 --- a/sdks/python/apache_beam/runners/portability/expansion_service_main.py +++ b/sdks/python/apache_beam/runners/portability/expansion_service_main.py @@ -55,7 +55,7 @@ def main(argv): with fully_qualified_named_transform.FullyQualifiedNamedTransform.with_filter( known_args.fully_qualified_name_glob): - address = '[::]:{}'.format(known_args.port) + address = 'localhost:{}'.format(known_args.port) server = grpc.server(thread_pool_executor.shared_unbounded_instance()) if known_args.serve_loopback_worker: beam_fn_api_pb2_grpc.add_BeamFnExternalWorkerPoolServicer_to_server( diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py index d798e96d3aa3..e5c9e9c7ac89 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py @@ -620,6 +620,7 @@ def start_worker(self): stub = beam_fn_api_pb2_grpc.BeamFnExternalWorkerPoolStub( GRPCChannelFactory.insecure_channel( self._external_payload.endpoint.url)) + _LOGGER.info('self.control_address: %s' % self.control_address) control_descriptor = endpoints_pb2.ApiServiceDescriptor( url=self.control_address) response = stub.StartWorker( diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py index 043b00dc934b..efb27715cd82 100644 --- a/sdks/python/apache_beam/utils/subprocess_server.py +++ b/sdks/python/apache_beam/utils/subprocess_server.py @@ -335,21 +335,24 @@ def path_to_maven_jar( ]) @classmethod - def path_to_beam_jar( + def parse_gradle_target(cls, gradle_target, artifact_id=None): + gradle_package = gradle_target.strip(':').rsplit(':', 1)[0] + if not artifact_id: + artifact_id = 'beam-' + gradle_package.replace(':', '-') + return gradle_package, artifact_id + + @classmethod + def path_to_dev_beam_jar( cls, gradle_target, appendix=None, version=beam_version, artifact_id=None): - if gradle_target in cls._BEAM_SERVICES.replacements: - return cls._BEAM_SERVICES.replacements[gradle_target] - - gradle_package = gradle_target.strip(':').rsplit(':', 1)[0] - if not artifact_id: - artifact_id = 'beam-' + gradle_package.replace(':', '-') + gradle_package, artifact_id = cls.parse_gradle_target( + gradle_target, artifact_id) project_root = os.path.sep.join( os.path.abspath(__file__).split(os.path.sep)[:-5]) - local_path = os.path.join( + return os.path.join( project_root, gradle_package.replace(':', os.path.sep), 'build', @@ -359,6 +362,22 @@ def path_to_beam_jar( version.replace('.dev', ''), classifier='SNAPSHOT', appendix=appendix)) + + @classmethod + def path_to_beam_jar( + cls, + gradle_target, + appendix=None, + version=beam_version, + artifact_id=None): + if gradle_target in cls._BEAM_SERVICES.replacements: + return cls._BEAM_SERVICES.replacements[gradle_target] + + _, artifact_id = cls.parse_gradle_target(gradle_target, artifact_id) + project_root = os.path.sep.join( + os.path.abspath(__file__).split(os.path.sep)[:-5]) + local_path = cls.path_to_dev_beam_jar( + gradle_target, appendix, version, artifact_id) if os.path.exists(local_path): _LOGGER.info('Using pre-built snapshot at %s', local_path) return local_path diff --git a/sdks/python/apache_beam/yaml/examples/testing/examples_test.py b/sdks/python/apache_beam/yaml/examples/testing/examples_test.py index 109e98410852..ee35e3430766 100644 --- a/sdks/python/apache_beam/yaml/examples/testing/examples_test.py +++ b/sdks/python/apache_beam/yaml/examples/testing/examples_test.py @@ -20,6 +20,7 @@ import logging import os import random +import sys import unittest from typing import Any from typing import Callable @@ -29,6 +30,7 @@ from typing import Union from unittest import mock +import pytest import yaml import apache_beam as beam @@ -36,6 +38,7 @@ from apache_beam.examples.snippets.util import assert_matches_stdout from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.utils import subprocess_server from apache_beam.yaml import yaml_provider from apache_beam.yaml import yaml_transform from apache_beam.yaml.readme_test import TestEnvironment @@ -263,6 +266,30 @@ def test_yaml_example(self): actual += list(transform.outputs.values()) check_output(expected)(actual) + if 'deps' in pipeline_spec_file: + test_yaml_example = pytest.mark.no_xdist(test_yaml_example) + test_yaml_example = unittest.skipIf( + sys.platform == 'win32', "Github virtualenv permissions issues.")( + test_yaml_example) + # This test fails, with an import error, for some (but not all) cloud + # tox environments when run as a github action (not reproducible locally). + # Adding debugging makes the failure go away. All indications are that + # this is some testing environmental issue. + test_yaml_example = unittest.skipIf( + '-cloud' in os.environ.get('TOX_ENV_NAME', ''), + 'Github actions environment issue.')( + test_yaml_example) + + if 'java_deps' in pipeline_spec_file: + test_yaml_example = pytest.mark.xlang_sql_expansion_service( + test_yaml_example) + test_yaml_example = unittest.skipIf( + not os.path.exists( + subprocess_server.JavaJarServer.path_to_dev_beam_jar( + 'sdks:java:extensions:sql:expansion-service:shadowJar')), + "Requires expansion service jars.")( + test_yaml_example) + return test_yaml_example diff --git a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py index e26d6140af23..9a5d848486ca 100644 --- a/sdks/python/apache_beam/yaml/yaml_enrichment_test.py +++ b/sdks/python/apache_beam/yaml/yaml_enrichment_test.py @@ -24,6 +24,7 @@ from apache_beam import Row from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.yaml import yaml_provider from apache_beam.yaml.yaml_transform import YamlTransform @@ -59,6 +60,8 @@ def test_enrichment_with_bigquery(self): with mock.patch('apache_beam.yaml.yaml_enrichment.enrichment_transform', FakeEnrichmentTransform(enrichment_handler=handler, handler_config=config)): + # Force a reload to respect our mock. + yaml_provider.standard_providers.cache_clear() input_pcoll = p | 'CreateInput' >> beam.Create(input_data) result = input_pcoll | YamlTransform( f''' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 239d1f124525..eca6c5f46dd2 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -31,6 +31,7 @@ import shutil import subprocess import sys +import tempfile import urllib.parse import warnings from collections.abc import Callable @@ -40,6 +41,7 @@ from typing import Optional from typing import Union +import clonevirtualenv import docstring_parser import yaml @@ -150,7 +152,7 @@ def _affinity(self, other: "Provider"): else: return 0 - @functools.cache + @functools.cache # pylint: disable=method-cache-max-size-none def with_extra_dependencies(self, dependencies: Iterable[str]): result = self._with_extra_dependencies(dependencies) if not hasattr(result, 'to_json'): @@ -451,7 +453,7 @@ def _affinity(self, other: "Provider"): def _with_extra_dependencies(self, dependencies: Iterable[str]): return ExternalPythonProvider( - self._urns, None, set(self._packages) + set(dependencies)) + self._urns, None, set(self._packages).union(set(dependencies))) @ExternalProvider.register_provider_type('yaml') @@ -642,11 +644,11 @@ def requires_inputs(self, typ, args): return super().requires_inputs(typ, args) def _with_extra_dependencies(self, dependencies): - external_provider = ExternalPythonProvider( # + external_provider = ExternalPythonProvider( # disable yapf { - typ: 'apache_beam.yaml.yaml_provider.standard_inline_providers.' - + typ.replace('-', '_') - for typ in self._transform_factories.keys() + typ: 'apache_beam.yaml.yaml_provider.standard_inline_providers.' + + typ.replace('-', '_') + for typ in self._transform_factories.keys() }, '__inline__', dependencies) @@ -1120,7 +1122,14 @@ class PypiExpansionService: """Expands transforms by fully qualified name in a virtual environment with the given dependencies. """ - VENV_CACHE = os.path.expanduser("~/.apache_beam/cache/venvs") + if 'TOX_WORK_DIR' in os.environ: + VENV_CACHE = tempfile.mkdtemp( + prefix='test-venv-cache-', dir=os.environ['TOX_WORK_DIR']) + elif 'RUNNER_WORKDIR' in os.environ: + VENV_CACHE = tempfile.mkdtemp( + prefix='test-venv-cache-', dir=os.environ['RUNNER_WORKDIR']) + else: + VENV_CACHE = os.path.expanduser("~/.apache_beam/cache/venvs") def __init__( self, packages: Iterable[str], base_python: str = sys.executable): @@ -1182,10 +1191,7 @@ def _create_venv_from_clone( if not os.path.exists(venv): try: clonable_venv = cls._create_venv_to_clone(base_python) - clonable_python = os.path.join(clonable_venv, 'bin', 'python') - subprocess.run( - [clonable_python, '-m', 'clonevirtualenv', clonable_venv, venv], - check=True) + clonevirtualenv.clone_virtualenv(clonable_venv, venv) venv_pip = os.path.join(venv, 'bin', 'pip') subprocess.run([venv_pip, 'install'] + packages, check=True) with open(venv + '-requirements.txt', 'w') as fout: @@ -1475,8 +1481,7 @@ def __getattr__(self, name): for provider in standard_providers()[typ]: if isinstance(provider, InlineProvider): return provider._transform_factories[typ] - else: - raise ValueError(f"No inline provider found for {name}") + raise ValueError(f"No inline provider found for {name}") standard_inline_providers = _InlineProviderNamespace() diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py index 75c5bd830940..744cbe6a8925 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform.py +++ b/sdks/python/apache_beam/yaml/yaml_transform.py @@ -716,7 +716,7 @@ def extract_extra_dependencies(spec): if not deps: return [], spec if not isinstance(deps, list): - raise TypeErrorError(f'Dependencies must be a list of strings, got {deps}') + raise TypeError(f'Dependencies must be a list of strings, got {deps}') return deps, dict( spec, config={k: v for k, v in spec['config'].items() if k != 'dependencies'}) diff --git a/sdks/python/setup.py b/sdks/python/setup.py index 9a45d287d49d..49d9ac368811 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -401,7 +401,8 @@ def get_portability_package_data(): 'docstring-parser>=0.15,<1.0', 'docutils>=0.18.1', 'pandas<2.3.0', - 'openai' + 'openai', + 'virtualenv-clone>=0.5,<1.0', ], 'test': [ 'docstring-parser>=0.15,<1.0', @@ -424,6 +425,7 @@ def get_portability_package_data(): 'testcontainers[mysql]>=3.0.3,<4.0.0', 'cryptography>=41.0.2', 'hypothesis>5.0.0,<7.0.0', + 'virtualenv-clone>=0.5,<1.0', ], 'gcp': [ 'cachetools>=3.1.0,<6', From c088abe8b87e403721a26706f5ad25d9dc58d130 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Tue, 18 Mar 2025 18:44:28 -0400 Subject: [PATCH 13/48] [AnomalyDetection] Support offline detectors (#34311) * Add offline detctors. * Fix typo. --- .../ml/anomaly/detectors/offline.py | 59 ++++++ .../apache_beam/ml/anomaly/transforms.py | 98 ++++++++- .../apache_beam/ml/anomaly/transforms_test.py | 197 ++++++++++++++++++ 3 files changed, 348 insertions(+), 6 deletions(-) create mode 100644 sdks/python/apache_beam/ml/anomaly/detectors/offline.py diff --git a/sdks/python/apache_beam/ml/anomaly/detectors/offline.py b/sdks/python/apache_beam/ml/anomaly/detectors/offline.py new file mode 100644 index 000000000000..6b8912575977 --- /dev/null +++ b/sdks/python/apache_beam/ml/anomaly/detectors/offline.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. +# + +from typing import Any +from typing import Dict +from typing import Optional + +import apache_beam as beam +from apache_beam.ml.anomaly.base import AnomalyDetector +from apache_beam.ml.anomaly.specifiable import specifiable +from apache_beam.ml.inference.base import KeyedModelHandler + + +@specifiable +class OfflineDetector(AnomalyDetector): + """A offline anomaly detector that uses a provided model handler for scoring. + + Args: + keyed_model_handler: The model handler to use for inference. + Requires a `KeyModelHandler[Any, Row, float, Any]` instance. + run_inference_args: Optional arguments to pass to RunInference + **kwargs: Additional keyword arguments to pass to the base + AnomalyDetector class. + """ + def __init__( + self, + keyed_model_handler: KeyedModelHandler[Any, beam.Row, float, Any], + run_inference_args: Optional[Dict[str, Any]] = None, + **kwargs): + super().__init__(**kwargs) + + # TODO: validate the model handler type + self._keyed_model_handler = keyed_model_handler + self._run_inference_args = run_inference_args or {} + + # always override model_identifier with model_id from the detector + self._run_inference_args["model_identifier"] = self._model_id + + def learn_one(self, x: beam.Row) -> None: + """Not implemented since OfflineDetector invokes RunInference directly.""" + raise NotImplementedError + + def score_one(self, x: beam.Row) -> Optional[float]: + """Not implemented since OfflineDetector invokes RunInference directly.""" + raise NotImplementedError diff --git a/sdks/python/apache_beam/ml/anomaly/transforms.py b/sdks/python/apache_beam/ml/anomaly/transforms.py index 08b656072ac8..d0a3ade605bc 100644 --- a/sdks/python/apache_beam/ml/anomaly/transforms.py +++ b/sdks/python/apache_beam/ml/anomaly/transforms.py @@ -17,9 +17,11 @@ import dataclasses import uuid +from typing import Any from typing import Callable from typing import Dict from typing import Iterable +from typing import List from typing import Optional from typing import Tuple from typing import TypeVar @@ -33,8 +35,10 @@ from apache_beam.ml.anomaly.base import AnomalyResult from apache_beam.ml.anomaly.base import EnsembleAnomalyDetector from apache_beam.ml.anomaly.base import ThresholdFn +from apache_beam.ml.anomaly.detectors.offline import OfflineDetector from apache_beam.ml.anomaly.specifiable import Spec from apache_beam.ml.anomaly.specifiable import Specifiable +from apache_beam.ml.inference.base import RunInference from apache_beam.transforms.userstate import ReadModifyWriteStateSpec KeyT = TypeVar('KeyT') @@ -97,9 +101,11 @@ def process( yield k1, (k2, AnomalyResult( example=data, - predictions=[AnomalyPrediction( - model_id=self._underlying._model_id, - score=self.score_and_learn(data))])) + predictions=[ + AnomalyPrediction( + model_id=self._underlying._model_id, + score=self.score_and_learn(data)) + ])) model_state.write(self._underlying) @@ -325,7 +331,8 @@ def expand( if self._aggregation_fn is None: # simply put predictions into an iterable (list) ret = ( - post_gbk | beam.MapTuple( + post_gbk + | beam.MapTuple( lambda k, v: ( k[0], @@ -353,7 +360,8 @@ def expand( # We use (original_key, temp_key) as the key for GroupByKey() so that # scores from multiple detectors per data point are grouped. ret = ( - post_gbk | beam.MapTuple( + post_gbk + | beam.MapTuple( lambda k, v, agg=aggregation_fn: ( @@ -406,6 +414,76 @@ def expand( return ret +class RunOfflineDetector(beam.PTransform[beam.PCollection[KeyedInputT], + beam.PCollection[KeyedOutputT]]): + """Runs a offline anomaly detector on a PCollection of data. + + This PTransform applies a `OfflineDetector` to the input data, handling + custom input/output conversion and inference. + + Args: + offline_detector: The `OfflineDetector` to run. + """ + def __init__(self, offline_detector: OfflineDetector): + self._offline_detector = offline_detector + + def unnest_and_convert( + self, nested: Tuple[Tuple[Any, Any], dict[str, List]]) -> KeyedOutputT: + """Unnests and converts the model output to AnomalyResult. + + Args: + nested: A tuple containing the combined key (origin key, temp key) and + a dictionary of input and output from RunInference. + + Returns: + A tuple containing the original key and AnomalyResult. + """ + key, value_dict = nested + score = value_dict['output'][0] + result = AnomalyResult( + example=value_dict['input'][0], + predictions=[ + AnomalyPrediction( + model_id=self._offline_detector._model_id, score=score) + ]) + return key[0], (key[1], result) + + def expand( + self, + input: beam.PCollection[KeyedInputT]) -> beam.PCollection[KeyedOutputT]: + model_uuid = f"{self._offline_detector._model_id}:{uuid.uuid4().hex[:6]}" + + # Call RunInference Transform with the keyed model handler + run_inference = RunInference( + self._offline_detector._keyed_model_handler, + **self._offline_detector._run_inference_args) + + # ((orig_key, temp_key), beam.Row) + rekeyed_model_input = input | "Rekey" >> beam.Map( + lambda x: ((x[0], x[1][0]), x[1][1])) + + # ((orig_key, temp_key), float) + rekeyed_model_output = ( + rekeyed_model_input + | f"Call RunInference ({model_uuid})" >> run_inference) + + # ((orig_key, temp_key), {'input':[row], 'output:[float]}) + rekeyed_cogbk = { + 'input': rekeyed_model_input, 'output': rekeyed_model_output + } | beam.CoGroupByKey() + + ret = ( + rekeyed_cogbk | + "Unnest and convert model output" >> beam.Map(self.unnest_and_convert)) + + if self._offline_detector._threshold_criterion: + ret = ( + ret | f"Run Threshold Criterion ({model_uuid})" >> + RunThresholdCriterion(self._offline_detector._threshold_criterion)) + + return ret + + class RunEnsembleDetector(beam.PTransform[beam.PCollection[KeyedInputT], beam.PCollection[KeyedOutputT]]): """Runs an ensemble of anomaly detectors on a PCollection of data. @@ -432,8 +510,14 @@ def expand( for idx, detector in enumerate(self._ensemble_detector._sub_detectors): if isinstance(detector, EnsembleAnomalyDetector): results.append( - input | f"Run Ensemble Detector at index {idx} ({model_uuid})" >> + input + | f"Run Ensemble Detector at index {idx} ({model_uuid})" >> RunEnsembleDetector(detector)) + elif isinstance(detector, OfflineDetector): + results.append( + input + | f"Run Offline Detector at index {idx} ({model_uuid})" >> + RunOfflineDetector(detector)) else: results.append( input @@ -518,6 +602,8 @@ def expand( if isinstance(self._root_detector, EnsembleAnomalyDetector): keyed_output = (keyed_input | RunEnsembleDetector(self._root_detector)) + elif isinstance(self._root_detector, OfflineDetector): + keyed_output = (keyed_input | RunOfflineDetector(self._root_detector)) else: keyed_output = (keyed_input | RunOneDetector(self._root_detector)) diff --git a/sdks/python/apache_beam/ml/anomaly/transforms_test.py b/sdks/python/apache_beam/ml/anomaly/transforms_test.py index cf398728f372..b8ed7c7e5e19 100644 --- a/sdks/python/apache_beam/ml/anomaly/transforms_test.py +++ b/sdks/python/apache_beam/ml/anomaly/transforms_test.py @@ -17,20 +17,45 @@ import logging import math +import os +import pickle +import shutil +import tempfile import unittest +from typing import Any +from typing import Dict from typing import Iterable +from typing import Optional +from typing import Sequence +from typing import SupportsFloat +from typing import Tuple + +import mock +import numpy +from sklearn.base import BaseEstimator import apache_beam as beam from apache_beam.ml.anomaly.aggregations import AnyVote from apache_beam.ml.anomaly.base import AnomalyPrediction from apache_beam.ml.anomaly.base import AnomalyResult from apache_beam.ml.anomaly.base import EnsembleAnomalyDetector +from apache_beam.ml.anomaly.detectors.offline import OfflineDetector from apache_beam.ml.anomaly.detectors.zscore import ZScore +from apache_beam.ml.anomaly.specifiable import Spec +from apache_beam.ml.anomaly.specifiable import Specifiable +from apache_beam.ml.anomaly.specifiable import _spec_type_to_subspace +from apache_beam.ml.anomaly.specifiable import specifiable from apache_beam.ml.anomaly.thresholds import FixedThreshold from apache_beam.ml.anomaly.thresholds import QuantileThreshold from apache_beam.ml.anomaly.transforms import AnomalyDetection from apache_beam.ml.anomaly.transforms import _StatefulThresholdDoFn from apache_beam.ml.anomaly.transforms import _StatelessThresholdDoFn +from apache_beam.ml.inference.base import KeyedModelHandler +from apache_beam.ml.inference.base import PredictionResult +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.inference.base import _PostProcessingModelHandler +from apache_beam.ml.inference.base import _PreProcessingModelHandler +from apache_beam.ml.inference.sklearn_inference import SklearnModelHandlerNumpy from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to @@ -253,6 +278,178 @@ def test_multiple_sub_detectors_with_aggregation(self): prediction in zip(self._input, aggregated)])) +class FakeNumpyModel(): + def __init__(self): + self.total_predict_calls = 0 + + def predict(self, input_vector: numpy.ndarray): + self.total_predict_calls += 1 + return [input_vector[0][0] * 10 - input_vector[0][1]] + + +def alternate_numpy_inference_fn( + model: BaseEstimator, + batch: Sequence[numpy.ndarray], + inference_args: Optional[Dict[str, Any]] = None) -> Any: + return [0] + + +def _to_keyed_numpy_array(t: Tuple[Any, beam.Row]): + """Converts an Apache Beam Row to a NumPy array.""" + return t[0], numpy.array(list(t[1])) + + +def _from_keyed_numpy_array(t: Tuple[Any, PredictionResult]): + assert isinstance(t[1].inference, SupportsFloat) + return t[0], float(t[1].inference) + + +class TestOfflineDetector(unittest.TestCase): + def setUp(self): + global SklearnModelHandlerNumpy, KeyedModelHandler + global _PreProcessingModelHandler, _PostProcessingModelHandler + # Make model handlers into Specifiable + SklearnModelHandlerNumpy = specifiable(SklearnModelHandlerNumpy) + KeyedModelHandler = specifiable(KeyedModelHandler) + _PreProcessingModelHandler = specifiable(_PreProcessingModelHandler) + _PostProcessingModelHandler = specifiable(_PostProcessingModelHandler) + self.tmpdir = tempfile.mkdtemp() + + def tearDown(self): + shutil.rmtree(self.tmpdir) + # Make the model handlers back to normal + SklearnModelHandlerNumpy.unspecifiable() + KeyedModelHandler.unspecifiable() + _PreProcessingModelHandler.unspecifiable() + _PostProcessingModelHandler.unspecifiable() + + def test_default_inference_fn(self): + temp_file_name = self.tmpdir + os.sep + 'pickled_file' + with open(temp_file_name, 'wb') as file: + pickle.dump(FakeNumpyModel(), file) + + keyed_model_handler = KeyedModelHandler( + SklearnModelHandlerNumpy(model_uri=temp_file_name)).with_preprocess_fn( + _to_keyed_numpy_array).with_postprocess_fn(_from_keyed_numpy_array) + + detector = OfflineDetector(keyed_model_handler=keyed_model_handler) + detector_spec = detector.to_spec() + expected_spec = Spec( + type='OfflineDetector', + config={ + 'keyed_model_handler': Spec( + type='_PostProcessingModelHandler', + config={ + 'base': Spec( + type='_PreProcessingModelHandler', + config={ + 'base': Spec( + type='KeyedModelHandler', + config={ + 'unkeyed': Spec( + type='SklearnModelHandlerNumpy', + config={'model_uri': temp_file_name}) + }), + 'preprocess_fn': Spec( + type='_to_keyed_numpy_array', config=None) + }), + 'postprocess_fn': Spec( + type='_from_keyed_numpy_array', config=None) + }) + }) + self.assertEqual(detector_spec, expected_spec) + + self.assertEqual(_spec_type_to_subspace('SklearnModelHandlerNumpy'), '*') + self.assertEqual(_spec_type_to_subspace('_PreProcessingModelHandler'), '*') + self.assertEqual(_spec_type_to_subspace('_PostProcessingModelHandler'), '*') + self.assertEqual(_spec_type_to_subspace('_to_keyed_numpy_array'), '*') + self.assertEqual(_spec_type_to_subspace('_from_keyed_numpy_array'), '*') + + # Make sure the spec from the detector can be used to reconstruct the same + # detector + detector_new = Specifiable.from_spec(detector_spec) + + input = [ + (1, beam.Row(x=1, y=2)), + (1, beam.Row(x=2, y=4)), + (1, beam.Row(x=3, y=6)), + ] + expected_predictions = [ + AnomalyPrediction( + model_id='OfflineDetector', + score=8.0, + label=None, + threshold=None, + info='', + source_predictions=None), + AnomalyPrediction( + model_id='OfflineDetector', + score=16.0, + label=None, + threshold=None, + info='', + source_predictions=None), + AnomalyPrediction( + model_id='OfflineDetector', + score=24.0, + label=None, + threshold=None, + info='', + source_predictions=None), + ] + with TestPipeline() as p: + result = ( + p | beam.Create(input) + # TODO: get rid of this conversion between BeamSchema to beam.Row. + | beam.Map(lambda t: (t[0], beam.Row(**t[1]._asdict()))) + | AnomalyDetection(detector_new)) + + assert_that( + result, + equal_to([( + input[0], + AnomalyResult(example=input[1], predictions=[prediction])) + for input, + prediction in zip(input, expected_predictions)])) + + def test_run_inference_args(self): + model_handler = SklearnModelHandlerNumpy(model_uri="unused") + detector = OfflineDetector( + keyed_model_handler=model_handler, + run_inference_args={"inference_args": { + "multiplier": 10 + }}) + + p = TestPipeline() + + input = [ + (1, beam.Row(x=1, y=2)), + (1, beam.Row(x=2, y=4)), + (1, beam.Row(x=3, y=6)), + ] + + # patch the RunInference in "apache_beam.ml.anomaly.transforms" where + # it is imported and call + with mock.patch('apache_beam.ml.anomaly.transforms.RunInference') as mock_run_inference: # pylint: disable=line-too-long + # make the actual RunInference as the sideeffect, so we record the call + # information but also create the true RunInference instance. + mock_run_inference.side_effect = RunInference + try: + p = TestPipeline() + _ = (p | beam.Create(input) | AnomalyDetection(detector)) + except: # pylint: disable=bare-except + pass + call_args = mock_run_inference.call_args[1] + self.assertEqual( + call_args, + { + 'inference_args': { + 'multiplier': 10 + }, + 'model_identifier': 'OfflineDetector' + }) + + R = beam.Row(x=10, y=20) From 00d9f00c562c3228ac52afccf89e694b64f0b051 Mon Sep 17 00:00:00 2001 From: Naireen Hussain Date: Tue, 18 Mar 2025 20:33:26 -0400 Subject: [PATCH 14/48] add option to disable metrics (#34303) Co-authored-by: Naireen --- .../org/apache/beam/sdk/io/kafka/KafkaIOInitializer.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 index 58aa3038844e..8bd9b234122b 100644 --- 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 @@ -19,6 +19,7 @@ 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. */ @@ -26,6 +27,8 @@ public class KafkaIOInitializer implements JvmInitializer { @Override public void beforeProcessing(PipelineOptions options) { - KafkaSinkMetrics.setSupportKafkaMetrics(true); + if (!ExperimentalOptions.hasExperiment(options, "disable_kafka_metrics")) { + KafkaSinkMetrics.setSupportKafkaMetrics(true); + } } } From b7ec7907e08f135ea7a062c5b0c0cc9765cac7c1 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Mon, 17 Mar 2025 20:47:30 +0000 Subject: [PATCH 15/48] initial schema transform provider implementation for SQS read functionality. --- .../sqs/providers/SqsReadConfiguration.java | 70 ++++++++++ .../SqsReadSchemaTransformProvider.java | 123 ++++++++++++++++++ 2 files changed, 193 insertions(+) create mode 100644 sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java create mode 100644 sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java new file mode 100644 index 000000000000..2266b8ac7b46 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java @@ -0,0 +1,70 @@ +/* + * 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.aws2.sqs.providers; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import software.amazon.awssdk.regions.Region; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class SqsReadConfiguration { + + public static Builder builder() { + return new AutoValue_SqsReadConfiguration.Builder() + .setMaxReadTimeSecs(null) + .setMaxNumRecords(Long.MAX_VALUE) + .setRegion(Region.AWS_GLOBAL.id()); + } + + @Nullable + public abstract String getRegion(); + + public Region region() { + return Optional.ofNullable(getRegion()).map(Region::of).orElse(Region.AWS_GLOBAL); + } + + public abstract String getQueueUrl(); + + @Nullable + public abstract Long getMaxReadTimeSecs(); + + @Nullable + public abstract Long getMaxNumRecords(); + + public long maxNumRecords() { + return Optional.ofNullable(getMaxNumRecords()).orElse(Long.MAX_VALUE); + } + + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setRegion(String region); + + public abstract Builder setQueueUrl(String queueUrl); + + public abstract Builder setMaxReadTimeSecs(@Nullable Long maxReadTimeSecs); + + public abstract Builder setMaxNumRecords(Long maxNumRecords); + + public abstract SqsReadConfiguration build(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java new file mode 100644 index 000000000000..350198b44ca6 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java @@ -0,0 +1,123 @@ +/* + * 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.aws2.sqs.providers; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import java.util.List; +import org.apache.beam.sdk.io.aws2.sqs.SqsIO; +import org.apache.beam.sdk.io.aws2.sqs.SqsMessage; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Duration; + +@AutoService(SchemaTransformProvider.class) +public class SqsReadSchemaTransformProvider + extends TypedSchemaTransformProvider { + + private static final String OUTPUT_TAG = "output"; + + public static final Schema OUTPUT_ROW_SCHEMA = + Schema.builder() + .addNullableStringField("body") + .addNullableStringField("message_id") + .addNullableStringField("receipt_handle") + .addNullableInt64Field("timestamp") + .addNullableInt64Field("request_timestamp") + .build(); + + @Override + protected SchemaTransform from(SqsReadConfiguration configuration) { + return new SqsReadSchemaTransform(configuration); + } + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:aws:sqs_read:v1"; + } + + @Override + public String description() { + return "Expose the SQS read functionality implemented in the Java SDK."; + } + + @Override + public List outputCollectionNames() { + return ImmutableList.of(OUTPUT_TAG); + } + + private static class SqsReadSchemaTransform extends SchemaTransform { + + private final SqsReadConfiguration configuration; + + SqsReadSchemaTransform(SqsReadConfiguration config) { + this.configuration = config; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + checkArgument( + input.getAll().isEmpty(), + String.format("Input to %s should be empty but it is not.", getClass().getSimpleName())); + + SqsIO.Read sqsRead = + SqsIO.read() + .withQueueUrl(configuration.getQueueUrl()) + .withMaxNumRecords(configuration.maxNumRecords()); + + Long maxReadtimeSecs = configuration.getMaxReadTimeSecs(); + + if (maxReadtimeSecs != null) { + sqsRead = sqsRead.withMaxReadTime(Duration.standardSeconds(maxReadtimeSecs)); + } + + return PCollectionRowTuple.of( + OUTPUT_TAG, + input + .getPipeline() + .apply("ReadFromSqs", sqsRead) + .apply( + "SqsMessageToRow", + MapElements.into(TypeDescriptors.rows()).via(new SqsMessageToBeamRow())) + .setRowSchema(OUTPUT_ROW_SCHEMA)); + } + } + + public static class SqsMessageToBeamRow implements SerializableFunction { + + @Override + public Row apply(SqsMessage input) { + return Row.withSchema(OUTPUT_ROW_SCHEMA) + .withFieldValue("body", input.getBody()) + .withFieldValue("message_id", input.getMessageId()) + .withFieldValue("receipt_handle", input.getReceiptHandle()) + .withFieldValue("timestamp", input.getTimeStamp()) + .withFieldValue("request_timestamp", input.getRequestTimeStamp()) + .build(); + } + } +} From e0df796748faf6e7d2bf070b4590587ed343792b Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Mon, 17 Mar 2025 22:28:04 +0000 Subject: [PATCH 16/48] fixing checkstyle errors. --- .../sdk/io/aws2/sqs/providers/SqsReadConfiguration.java | 1 + .../sqs/providers/SqsReadSchemaTransformProvider.java | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java index 2266b8ac7b46..89daca86f3c6 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import software.amazon.awssdk.regions.Region; +/** Configuration class for reading data from an AWS SQS queue. */ @DefaultSchema(AutoValueSchema.class) @AutoValue public abstract class SqsReadConfiguration { diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java index 350198b44ca6..91b52583f87f 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java @@ -35,6 +35,14 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; +/** + * An implementation of {@link TypedSchemaTransformProvider} for jobs reading data from AWS SQS + * queues and configured via {@link SqsReadConfiguration}. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ @AutoService(SchemaTransformProvider.class) public class SqsReadSchemaTransformProvider extends TypedSchemaTransformProvider { From ea7980fe6ec161c831ec83ae460367f52629fca0 Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 18 Mar 2025 01:11:22 +0000 Subject: [PATCH 17/48] removing region configuration since its not needed --- .../io/aws2/sqs/providers/SqsReadConfiguration.java | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java index 89daca86f3c6..1ed779b6b228 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadConfiguration.java @@ -22,7 +22,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import software.amazon.awssdk.regions.Region; /** Configuration class for reading data from an AWS SQS queue. */ @DefaultSchema(AutoValueSchema.class) @@ -32,15 +31,7 @@ public abstract class SqsReadConfiguration { public static Builder builder() { return new AutoValue_SqsReadConfiguration.Builder() .setMaxReadTimeSecs(null) - .setMaxNumRecords(Long.MAX_VALUE) - .setRegion(Region.AWS_GLOBAL.id()); - } - - @Nullable - public abstract String getRegion(); - - public Region region() { - return Optional.ofNullable(getRegion()).map(Region::of).orElse(Region.AWS_GLOBAL); + .setMaxNumRecords(Long.MAX_VALUE); } public abstract String getQueueUrl(); @@ -58,8 +49,6 @@ public long maxNumRecords() { @AutoValue.Builder public abstract static class Builder { - public abstract Builder setRegion(String region); - public abstract Builder setQueueUrl(String queueUrl); public abstract Builder setMaxReadTimeSecs(@Nullable Long maxReadTimeSecs); From db4c87dbea3394a34cfd64223f4704916c3ac21c Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 18 Mar 2025 01:23:08 +0000 Subject: [PATCH 18/48] adding package info for the new providers --- .../io/aws2/sqs/providers/package-info.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) create mode 100644 sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/package-info.java diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/package-info.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/package-info.java new file mode 100644 index 000000000000..094d55039480 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Defines external schema transformation providers for Amazon Web Services SQS. */ +package org.apache.beam.sdk.io.aws2.sqs.providers; From 2392a7d116264937209794c111c4d24776311f5b Mon Sep 17 00:00:00 2001 From: Pablo Rodriguez Defino Date: Tue, 18 Mar 2025 21:18:24 +0000 Subject: [PATCH 19/48] added tests to the provider --- .../SqsReadSchemaTransformProvider.java | 2 +- .../SqsReadSchemaTransformProviderTest.java | 144 ++++++++++++++++++ 2 files changed, 145 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProviderTest.java diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java index 91b52583f87f..a381ccf96022 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProvider.java @@ -47,7 +47,7 @@ public class SqsReadSchemaTransformProvider extends TypedSchemaTransformProvider { - private static final String OUTPUT_TAG = "output"; + public static final String OUTPUT_TAG = "output"; public static final Schema OUTPUT_ROW_SCHEMA = Schema.builder() diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProviderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProviderTest.java new file mode 100644 index 000000000000..4a720e68650b --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/providers/SqsReadSchemaTransformProviderTest.java @@ -0,0 +1,144 @@ +/* + * 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.aws2.sqs.providers; + +import static java.util.stream.Collectors.toList; +import static java.util.stream.IntStream.range; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; +import static software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName.SENT_TIMESTAMP; +import static software.amazon.awssdk.services.sqs.model.QueueAttributeName.VISIBILITY_TIMEOUT; + +import java.util.List; +import java.util.function.Consumer; +import org.apache.beam.sdk.io.aws2.MockClientBuilderFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.SqsClientBuilder; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; +import software.amazon.awssdk.services.sqs.model.GetQueueAttributesResponse; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; + +@RunWith(MockitoJUnitRunner.class) +public class SqsReadSchemaTransformProviderTest { + @Rule public TestPipeline p = TestPipeline.create(); + @Mock public SqsClient sqs; + + @Before + public void configureClientBuilderFactory() { + MockClientBuilderFactory.set(p, SqsClientBuilder.class, sqs); + + when(sqs.getQueueAttributes(any(Consumer.class))) + .thenReturn( + GetQueueAttributesResponse.builder() + .attributes(ImmutableMap.of(VISIBILITY_TIMEOUT, "600")) + .build()); + } + + @Test(expected = IllegalArgumentException.class) + public void testBuildTransformWithRowRequiredNotPresent() { + SqsReadSchemaTransformProvider provider = new SqsReadSchemaTransformProvider(); + + Row.withSchema(provider.configurationSchema()).withFieldValue("max_num_records", 10L).build(); + } + + @Test + public void testBuildTransformWithRow() { + SqsReadSchemaTransformProvider provider = new SqsReadSchemaTransformProvider(); + + Row transformConfigRow = + Row.withSchema(provider.configurationSchema()) + .withFieldValue("queue_url", "https://somedummy.url") + .withFieldValue("max_read_time_secs", 10L) + .build(); + + provider.from(transformConfigRow); + } + + @Test + public void testReadOnce() { + List expected = range(0, 10).mapToObj(this::message).collect(toList()); + + when(sqs.receiveMessage(any(ReceiveMessageRequest.class))) + .thenReturn( + ReceiveMessageResponse.builder().messages(expected).build(), + ReceiveMessageResponse.builder().build()); + + ArgumentCaptor deleteReq = + ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); + when(sqs.deleteMessageBatch(deleteReq.capture())) + .thenReturn(DeleteMessageBatchResponse.builder().build()); + + SqsReadSchemaTransformProvider provider = new SqsReadSchemaTransformProvider(); + + Row readConfig = + Row.withSchema(provider.configurationSchema()) + .withFieldValue("queue_url", "https://somedummy.url") + .withFieldValue("max_read_time_secs", 10L) + .build(); + + PCollection result = + PCollectionRowTuple.empty(p) + .apply(provider.from(readConfig)) + .get(SqsReadSchemaTransformProvider.OUTPUT_TAG) + .apply(ParDo.of(new ToMessage())); + + // all expected messages are read + PAssert.that(result).containsInAnyOrder(expected); + p.run(); + } + + private Message message(int i) { + return Message.builder() + .messageId("id" + i) + .body("body" + i) + .receiptHandle("handle" + i) + .attributes(ImmutableMap.of(SENT_TIMESTAMP, Integer.toString(i))) + .build(); + } + + static class ToMessage extends DoFn { + @ProcessElement + public void processElement(@Element Row msg, OutputReceiver out) { + out.output( + Message.builder() + .messageId(msg.getString("message_id")) + .body(msg.getString("body")) + .receiptHandle(msg.getString("receipt_handle")) + .attributes(ImmutableMap.of(SENT_TIMESTAMP, msg.getInt64("timestamp").toString())) + .build()); + } + } +} From 76a1374f423d849979541df14f3d2b43684a663b Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Wed, 19 Mar 2025 03:50:18 -0700 Subject: [PATCH 20/48] [Dataflow Streaming] WindmillTimerInternals: Use a single map to store timer data + liveness (#34292) --- .../worker/WindmillTimerInternals.java | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java index 9b166ee1584b..c8143cae864d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java @@ -21,6 +21,10 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.io.IOException; +import java.util.AbstractMap.SimpleEntry; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; import java.util.function.Consumer; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; @@ -35,9 +39,6 @@ import org.apache.beam.sdk.util.VarInt; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; 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.collect.HashBasedTable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table.Cell; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; @@ -64,10 +65,9 @@ class WindmillTimerInternals implements TimerInternals { // though technically in Windmill this is only enforced per ID and namespace // and TimeDomain. This TimerInternals is scoped to a step and key, shared // across namespaces. - private final Table timers = HashBasedTable.create(); - - // Map from timer id to whether it is to be deleted or set - private final Table timerStillPresent = HashBasedTable.create(); + private final Map< + Entry, Entry> + timerMap = new HashMap<>(); private final Watermarks watermarks; private final Instant processingTime; @@ -96,8 +96,9 @@ public WindmillTimerInternals withPrefix(WindmillNamespacePrefix prefix) { @Override public void setTimer(TimerData timerKey) { String timerDataKey = getTimerDataKey(timerKey.getTimerId(), timerKey.getTimerFamilyId()); - timers.put(timerDataKey, timerKey.getNamespace(), timerKey); - timerStillPresent.put(timerDataKey, timerKey.getNamespace(), true); + timerMap.put( + new SimpleEntry<>(timerDataKey, timerKey.getNamespace()), + new SimpleEntry<>(timerKey, true)); onTimerModified.accept(timerKey); } @@ -126,8 +127,9 @@ private static String getTimerDataKey(String timerId, String timerFamilyId) { @Override public void deleteTimer(TimerData timerKey) { String timerDataKey = getTimerDataKey(timerKey.getTimerId(), timerKey.getTimerFamilyId()); - timers.put(timerDataKey, timerKey.getNamespace(), timerKey); - timerStillPresent.put(timerDataKey, timerKey.getNamespace(), false); + timerMap.put( + new SimpleEntry<>(timerDataKey, timerKey.getNamespace()), + new SimpleEntry<>(timerKey, false)); onTimerModified.accept(timerKey.deleted()); } @@ -189,16 +191,16 @@ public Instant currentInputWatermarkTime() { } public void persistTo(Windmill.WorkItemCommitRequest.Builder outputBuilder) { - for (Cell cell : timerStillPresent.cellSet()) { + for (Entry value : timerMap.values()) { // Regardless of whether it is set or not, it must have some TimerData stored so we // can know its time domain - TimerData timerData = timers.get(cell.getRowKey(), cell.getColumnKey()); + TimerData timerData = value.getKey(); Timer.Builder timer = buildWindmillTimerFromTimerData( stateFamily, prefix, timerData, outputBuilder.addOutputTimersBuilder()); - if (cell.getValue()) { + if (value.getValue()) { // Setting the timer. If it is a user timer, set a hold. // Only set a hold if it's needed and if the hold is before the end of the global window. @@ -241,7 +243,7 @@ public void persistTo(Windmill.WorkItemCommitRequest.Builder outputBuilder) { } // Wipe the unpersisted state - timers.clear(); + timerMap.clear(); } private boolean needsWatermarkHold(TimerData timerData) { From 16fb2fbabcd8faabf60af02c144592116c29413d Mon Sep 17 00:00:00 2001 From: scwhittle Date: Wed, 19 Mar 2025 11:50:37 +0100 Subject: [PATCH 21/48] add equals hashCode to BoundedToUnboundedSourceAdapter (#34057) --- .../UnboundedReadFromBoundedSource.java | 15 +++++++++++++++ .../UnboundedReadFromBoundedSourceTest.java | 5 ++--- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java index 430cec0d2479..7256be139e72 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSource.java @@ -28,6 +28,7 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -121,6 +122,20 @@ public BoundedToUnboundedSourceAdapter(BoundedSource boundedSource) { this.boundedSource = boundedSource; } + @Override + public boolean equals(Object other) { + if (!(other instanceof BoundedToUnboundedSourceAdapter)) { + return false; + } + return Objects.equals( + boundedSource, ((BoundedToUnboundedSourceAdapter) other).boundedSource); + } + + @Override + public int hashCode() { + return Objects.hash(boundedSource); + } + @Override public void validate() { boundedSource.validate(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java index af5a442d1523..3279f66d5c6f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util.construction; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -262,7 +261,7 @@ public void testInvokesSplitWithDefaultNumSplitsTooLarge() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); List splits = unboundedCountingSource.split(100, options); assertEquals(1, splits.size()); - assertNotEquals(splits.get(0), unboundedCountingSource); + assertEquals(splits.get(0), unboundedCountingSource); } @Test @@ -272,7 +271,7 @@ public void testInvokingSplitProducesAtLeastOneSplit() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); List splits = unboundedCountingSource.split(100, options); assertEquals(1, splits.size()); - assertNotEquals(splits.get(0), unboundedCountingSource); + assertEquals(splits.get(0), unboundedCountingSource); } @Test From fbead7f7e3b9aea80828aecaf472e34c64232e93 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 19 Mar 2025 08:46:27 -0400 Subject: [PATCH 22/48] Remove cancelled tasks from ReadOperation queue when shutting down (#34335) * Remove cancelled tasks from ReadOperation queue when shutting down * Try shutdowNow * spotless --- .../worker/util/common/worker/ReadOperation.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java index 1ee8f2bc843e..d6b020483d4c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java @@ -225,8 +225,16 @@ public void run() { if (!scheduler.isTerminated()) { LOG.error( "Failed to terminate periodic progress reporting in 1 minute. " - + "Waiting for it to terminate indefinitely..."); - scheduler.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + + "Waiting for it to terminate 10 minutes before forcing"); + scheduler.awaitTermination(10, TimeUnit.MINUTES); + if (!scheduler.isTerminated()) { + LOG.error( + "Failed to terminate periodic progress reporting in 10 " + + "minutes. Trying to force termination then waiting " + + "indefinitely..."); + scheduler.shutdownNow(); + scheduler.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + } LOG.info("Periodic progress reporting terminated."); } } From cb67604ccf3c43283c4fb77ddeb38428541648f0 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 19 Mar 2025 10:20:12 -0400 Subject: [PATCH 23/48] [Prism] Refactor stageState to a behavior interface to reduce branch combinatorics (#34132) * [Prism] Refactor stageState to a behavior interface to reduce branch combinatorics * Revert on demand pending change. * revert attempt at lock avoidance, causes hangs. --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- runners/prism/java/build.gradle | 4 +- .../prism/internal/engine/elementmanager.go | 455 +++++++++++------- 2 files changed, 285 insertions(+), 174 deletions(-) diff --git a/runners/prism/java/build.gradle b/runners/prism/java/build.gradle index ab620a2303f4..1d58a568c432 100644 --- a/runners/prism/java/build.gradle +++ b/runners/prism/java/build.gradle @@ -99,7 +99,9 @@ def sickbayTests = [ 'org.apache.beam.sdk.transforms.ViewTest.testTriggeredLatestSingleton', // Prism doesn't support multiple TestStreams. - 'org.apache.beam.sdk.testing.TestStreamTest.testMultipleStreams', + 'org.apache.beam.sdk.testing.TestStreamTest.testMultipleStreams', + // Sometimes fails missing a final 'AFTER'. Otherwise, Hangs in ElementManager.FailBundle due to a held stageState lock. + 'org.apache.beam.sdk.testing.TestStreamTest.testMultiStage', // GroupIntoBatchesTest tests that fail: // Teststream has bad KV encodings due to using an outer context. 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 d1873139d0d2..2aecd2991364 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -264,16 +264,18 @@ func (em *ElementManager) AddStage(ID string, inputIDs, outputIDs []string, side // means elements will only be processed based on windowing strategies. func (em *ElementManager) StageAggregates(ID string, strat WinStrat) { ss := em.stages[ID] - ss.aggregate = true + ss.kind = &aggregateStageKind{} ss.strat = strat + ss.inprogressKeys = set[string]{} } // StageStateful marks the given stage as stateful, which means elements are // processed by key. func (em *ElementManager) StageStateful(ID string, stateTypeLen map[LinkID]func([]byte) int) { ss := em.stages[ID] - ss.stateful = true + ss.kind = &statefulStageKind{} ss.stateTypeLen = stateTypeLen + ss.inprogressKeys = set[string]{} } // StageOnWindowExpiration marks the given stage as stateful, which means elements are @@ -818,16 +820,11 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol keyBytes = info.KeyDec(kbuf) // TODO: Optimize unnecessary copies. This is tripleteeing? } for _, w := range ws { - if stage.aggregate { - stage.mu.Lock() - pn = stage.state[LinkID{}][w][string(keyBytes)].Pane - stage.mu.Unlock() - } newPending = append(newPending, element{ window: w, timestamp: et, - pane: pn, + pane: stage.kind.updatePane(stage, pn, w, keyBytes), elmBytes: elmBytes, keyBytes: keyBytes, sequence: seq, @@ -1013,10 +1010,7 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag // FailBundle clears the extant data allowing the execution to shut down. func (em *ElementManager) FailBundle(rb RunBundle) { stage := em.stages[rb.StageID] - if !stage.mu.TryLock() { - slog.Error("failing bundle: engine state is corrupted with a locked mutex.", "bundle", rb) - return - } + stage.mu.Lock() completed := stage.inprogress[rb.BundleID] em.addPending(-len(completed.es)) delete(stage.inprogress, rb.BundleID) @@ -1122,8 +1116,7 @@ type stageState struct { sides []LinkID // PCollection IDs of side inputs that can block execution. // Special handling bits - stateful bool // whether this stage uses state or timers, and needs keyed processing. - aggregate bool // whether this stage needs to block for aggregation. + kind stageKind strat WinStrat // Windowing Strategy for aggregation fireings. processingTimeTimersFamilies map[string]bool // Indicates which timer families use the processing time domain. @@ -1161,6 +1154,51 @@ type stageState struct { processingTimeTimers *timerHandler } +// stageKind handles behavioral differences between ordinary, stateful, and aggregation stage kinds. +// +// kinds should be stateless, and stageState retains all state for the stage, +// even if it's unused by the current kind. +type stageKind interface { + // addPending handles adding new pending elements to the stage appropriate for the kind. + addPending(ss *stageState, em *ElementManager, newPending []element) int + // buildEventTimeBundle handles building bundles for the stage per it's kind. + buildEventTimeBundle(ss *stageState, watermark mtime.Time) (toProcess elementHeap, minTs mtime.Time, newKeys set[string], holdsInBundle map[mtime.Time]int, schedulable bool, pendingAdjustment int) + + // updatePane based on the stage state. + updatePane(ss *stageState, pane typex.PaneInfo, w typex.Window, keyBytes []byte) typex.PaneInfo +} + +// ordinaryStageKind represents stages that have no special behavior associated with them. +// This represents most batch pending elements, and doesn't require keyed elements. +type ordinaryStageKind struct{} + +func (*ordinaryStageKind) String() string { return "OrdinaryStage" } + +func (*ordinaryStageKind) updatePane(ss *stageState, pane typex.PaneInfo, w typex.Window, keyBytes []byte) typex.PaneInfo { + return pane +} + +// statefulStageKind require keyed elements, and handles stages with stateful transforms, with state and timers. +type statefulStageKind struct{} + +func (*statefulStageKind) String() string { return "StatefulStage" } + +func (*statefulStageKind) updatePane(ss *stageState, pane typex.PaneInfo, w typex.Window, keyBytes []byte) typex.PaneInfo { + return pane +} + +// aggregateStageKind handles stages that perform aggregations over their primary inputs. +// They are a specialized kind of stateful stage, that doesn't handle timers. +type aggregateStageKind struct{} + +func (*aggregateStageKind) String() string { return "AggregateStage" } + +func (*aggregateStageKind) updatePane(ss *stageState, pane typex.PaneInfo, w typex.Window, keyBytes []byte) typex.PaneInfo { + ss.mu.Lock() + defer ss.mu.Unlock() + return ss.state[LinkID{}][w][string(keyBytes)].Pane +} + // timerKey uniquely identifies a given timer within the space of a user key. type timerKey struct { family, tag string @@ -1186,6 +1224,7 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st sides: sides, state: map[LinkID]map[typex.Window]map[string]StateData{}, watermarkHolds: newHoldTracker(), + kind: &ordinaryStageKind{}, input: mtime.MinTimestamp, output: mtime.MinTimestamp, @@ -1208,98 +1247,122 @@ func makeStageState(ID string, inputIDs, outputIDs []string, sides []LinkID) *st func (ss *stageState) AddPending(em *ElementManager, newPending []element) int { ss.mu.Lock() defer ss.mu.Unlock() - if ss.aggregate { - // Late Data is data that has arrived after that window has expired. - // We only need to drop late data before aggregations. - // TODO - handle for side inputs too. - threshold := ss.output - origPending := make([]element, 0, ss.pending.Len()) - for _, e := range newPending { - if ss.strat.EarliestCompletion(e.window) < threshold { - continue - } - origPending = append(origPending, e) + return ss.kind.addPending(ss, em, newPending) +} + +// addPending for aggregate stages behaves likes stateful stages, but don't need to handle timers or a separate window +// expiration condition. +func (*aggregateStageKind) addPending(ss *stageState, em *ElementManager, newPending []element) int { + // Late Data is data that has arrived after that window has expired. + // We only need to drop late data before aggregations. + // TODO - handle for side inputs too. + threshold := ss.output + origPending := make([]element, 0, ss.pending.Len()) + for _, e := range newPending { + if ss.strat.EarliestCompletion(e.window) < threshold { + continue } - newPending = origPending + origPending = append(origPending, e) } - if ss.stateful || ss.aggregate { - if ss.pendingByKeys == nil { - ss.pendingByKeys = map[string]*dataAndTimers{} - } - count := 0 - for _, e := range newPending { - count++ - if len(e.keyBytes) == 0 { - panic(fmt.Sprintf("zero length key: %v %v", ss.ID, ss.inputID)) - } - dnt, ok := ss.pendingByKeys[string(e.keyBytes)] - if !ok { - dnt = &dataAndTimers{ - 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) + newPending = origPending - if e.IsTimer() { - if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { - // existing timer! - // don't increase the count this time, as "this" timer is already pending. - count-- - // clear out the existing hold for accounting purposes. - ss.watermarkHolds.Drop(lastSet.hold, 1) - } - // Update the last set time on the timer. - dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}] = timerTimes{firing: e.timestamp, hold: e.holdTimestamp} - - // Mark the hold in the heap. - ss.watermarkHolds.Add(e.holdTimestamp, 1) - } else if ss.aggregate { - // Check on triggers for this key. - // We use an empty linkID as the key into state for aggregations. - if ss.state == nil { - ss.state = make(map[LinkID]map[typex.Window]map[string]StateData) - } - lv, ok := ss.state[LinkID{}] - if !ok { - lv = make(map[typex.Window]map[string]StateData) - ss.state[LinkID{}] = lv - } - wv, ok := lv[e.window] - if !ok { - wv = make(map[string]StateData) - lv[e.window] = wv - } - state := wv[string(e.keyBytes)] - endOfWindowReached := e.window.MaxTimestamp() < ss.input - ready := ss.strat.IsTriggerReady(triggerInput{ - newElementCount: 1, - endOfWindowReached: endOfWindowReached, - }, &state) + if ss.pendingByKeys == nil { + ss.pendingByKeys = map[string]*dataAndTimers{} + } + count := 0 + for _, e := range newPending { + count++ + if len(e.keyBytes) == 0 { + panic(fmt.Sprintf("zero length key: %v %v", ss.ID, ss.inputID)) + } + dnt, ok := ss.pendingByKeys[string(e.keyBytes)] + if !ok { + dnt = &dataAndTimers{} + ss.pendingByKeys[string(e.keyBytes)] = dnt + } + heap.Push(&dnt.elements, e) + // Check on triggers for this key. + // We use an empty linkID as the key into state for aggregations. + if ss.state == nil { + ss.state = make(map[LinkID]map[typex.Window]map[string]StateData) + } + lv, ok := ss.state[LinkID{}] + if !ok { + lv = make(map[typex.Window]map[string]StateData) + ss.state[LinkID{}] = lv + } + wv, ok := lv[e.window] + if !ok { + wv = make(map[string]StateData) + lv[e.window] = wv + } + state := wv[string(e.keyBytes)] + endOfWindowReached := e.window.MaxTimestamp() < ss.input + ready := ss.strat.IsTriggerReady(triggerInput{ + newElementCount: 1, + endOfWindowReached: endOfWindowReached, + }, &state) + + if ready { + state.Pane = computeNextTriggeredPane(state.Pane, endOfWindowReached) + } + // Store the state as triggers may have changed it. + ss.state[LinkID{}][e.window][string(e.keyBytes)] = state - if ready { - state.Pane = computeNextTriggeredPane(state.Pane, endOfWindowReached) - } - // Store the state as triggers may have changed it. - ss.state[LinkID{}][e.window][string(e.keyBytes)] = state + // If we're ready, it's time to fire! + if ready { + count += ss.buildTriggeredBundle(em, e.keyBytes, e.window) + } + } + return count +} - // If we're ready, it's time to fire! - if ready { - count += ss.buildTriggeredBundle(em, e.keyBytes, e.window) +func (*statefulStageKind) addPending(ss *stageState, em *ElementManager, newPending []element) int { + if ss.pendingByKeys == nil { + ss.pendingByKeys = map[string]*dataAndTimers{} + } + count := 0 + for _, e := range newPending { + count++ + if len(e.keyBytes) == 0 { + panic(fmt.Sprintf("zero length key: %v %v", ss.ID, ss.inputID)) + } + dnt, ok := ss.pendingByKeys[string(e.keyBytes)] + if !ok { + dnt = &dataAndTimers{ + 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)) } } - return count + heap.Push(&dnt.elements, e) + + if e.IsTimer() { + if lastSet, ok := dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}]; ok { + // existing timer! + // don't increase the count this time, as "this" timer is already pending. + count-- + // clear out the existing hold for accounting purposes. + ss.watermarkHolds.Drop(lastSet.hold, 1) + } + // Update the last set time on the timer. + dnt.timers[timerKey{family: e.family, tag: e.tag, window: e.window}] = timerTimes{firing: e.timestamp, hold: e.holdTimestamp} + + // Mark the hold in the heap. + ss.watermarkHolds.Add(e.holdTimestamp, 1) + } } - // Default path. + return count +} + +func (*ordinaryStageKind) addPending(ss *stageState, em *ElementManager, newPending []element) int { ss.pending = append(ss.pending, newPending...) heap.Init(&ss.pending) return len(newPending) @@ -1509,14 +1572,26 @@ func (ss *stageState) startEventTimeBundle(watermark mtime.Time, genBundID func( }() ss.mu.Lock() defer ss.mu.Unlock() + toProcess, minTs, newKeys, holdsInBundle, stillSchedulable, accumulatingPendingAdjustment := ss.kind.buildEventTimeBundle(ss, watermark) + + if len(toProcess) == 0 { + // If we have nothing, there's nothing to progress. + return "", false, stillSchedulable, accumulatingPendingAdjustment + } + + bundID := ss.makeInProgressBundle(genBundID, toProcess, minTs, newKeys, holdsInBundle) + return bundID, true, stillSchedulable, accumulatingPendingAdjustment +} - var toProcess = ss.pending +// buildEventTimeBundle for ordinary stages processes all pending elements. +func (*ordinaryStageKind) buildEventTimeBundle(ss *stageState, watermark mtime.Time) (toProcess elementHeap, minTs mtime.Time, newKeys set[string], holdsInBundle map[mtime.Time]int, schedulable bool, pendingAdjustment int) { + toProcess = ss.pending ss.pending = nil + return toProcess, mtime.MaxTimestamp, nil, nil, true, 0 +} - heap.Init(&ss.pending) - if ss.inprogressKeys == nil { - ss.inprogressKeys = set[string]{} - } +// buildEventTimeBundle for stateful stages, processes all elements that are before the input watermark time. +func (*statefulStageKind) buildEventTimeBundle(ss *stageState, watermark mtime.Time) (toProcess elementHeap, _ mtime.Time, _ set[string], _ map[mtime.Time]int, schedulable bool, pendingAdjustment int) { minTs := mtime.MaxTimestamp // TODO: Allow configurable limit of keys per bundle, and elements per key to improve parallelism. // TODO: when we do, we need to ensure that the stage remains schedualable for bundle execution, for remaining pending elements and keys. @@ -1530,7 +1605,6 @@ func (ss *stageState) startEventTimeBundle(watermark mtime.Time, genBundID func( // timers might have held back the minimum pending watermark. timerCleared := false - accumulatingPendingAdjustment := 0 keysPerBundle: for k, dnt := range ss.pendingByKeys { if ss.inprogressKeys.present(k) { @@ -1560,31 +1634,6 @@ keysPerBundle: (!dataInBundle && !dnt.elements[0].IsTimer())) { // we're about to add data to a bundle that already has a time break } - // If this is an aggregation, only include elements for this key - // if we're after the end of window, or after the window expiry deadline. - if ss.aggregate { - // We will only ever trigger aggregations by watermark at most twice, once the watermark passes the window ends for OnTime completion, - // and once for when the window is closing. - elm := dnt.elements[0] - if watermark <= elm.window.MaxTimestamp() { - // The watermark hasn't passed the end of the window yet, we do nothing. - break - } - // Watermark is past the end of this window. Have we fired an OnTime pane yet? - state := ss.state[LinkID{}][elm.window][string(elm.keyBytes)] - // If this is not the ontime firing for this key. - - if state.Pane.Timing != typex.PaneEarly && watermark <= ss.strat.EarliestCompletion(elm.window) { - // The watermark is still before the earliest final completion for this window. - // Do not add further data for this firing. - // If this is the Never trigger, we also don't fire OnTime until after the earliest completion. - break - } - if ss.strat.IsNeverTrigger() && watermark <= ss.strat.EarliestCompletion(elm.window) { - // The NeverTrigger only has a single firing at the end of window + allowed lateness. - break - } - } e := heap.Pop(&dnt.elements).(element) if e.IsData() { dataInBundle = true @@ -1603,50 +1652,119 @@ keysPerBundle: delete(dnt.timers, timerKey{family: e.family, tag: e.tag, window: e.window}) } toProcessForKey = append(toProcessForKey, e) - if !ss.aggregate && OneElementPerKey { - // For aggregations, a single key is a single element. + if OneElementPerKey { break } } + toProcess = append(toProcess, toProcessForKey...) - // Get the pane for the aggregation correct, only mutate it once per key and window. - if ss.aggregate { - handledWindows := set[typex.Window]{} - for _, elm := range toProcessForKey { - state := ss.state[LinkID{}][elm.window][string(elm.keyBytes)] - if handledWindows.present(elm.window) { - // The pane is already correct for this key + window + firing. - if ss.strat.Accumulating && !state.Pane.IsLast { - // If this isn't the last pane, then we must add the element back to the pending store for subsequent firings. - heap.Push(&dnt.elements, elm) - accumulatingPendingAdjustment++ - } - continue - } - handledWindows.insert(elm.window) - - state.Pane = computeNextWatermarkPane(state.Pane) - // Determine if this is the last pane. - // Check if this is the post closing firing, which will be the last one. - // Unless it's the ontime pane, at which point it can never be last. - if watermark > ss.strat.EarliestCompletion(elm.window) && state.Pane.Timing != typex.PaneOnTime { - state.Pane.IsLast = true - } - if ss.strat.AllowedLateness == 0 || ss.strat.IsNeverTrigger() { - // If the allowed lateness is zero, then this will be the last pane. - // If this is the NeverTrigger, it's the last pane. - state.Pane.IsLast = true - } - ss.state[LinkID{}][elm.window][string(elm.keyBytes)] = state + if dnt.elements.Len() == 0 { + delete(ss.pendingByKeys, k) + } + if OneKeyPerBundle { + break keysPerBundle + } + } + + // If we're out of data, and timers were not cleared then the watermark is accurate. + stillSchedulable := !(len(ss.pendingByKeys) == 0 && !timerCleared) + + return toProcess, minTs, newKeys, holdsInBundle, stillSchedulable, 0 +} + +// buildEventTimeBundle for aggregation stages, processes all elements that are within the watermark for completed windows. +func (*aggregateStageKind) buildEventTimeBundle(ss *stageState, watermark mtime.Time) (toProcess elementHeap, _ mtime.Time, _ set[string], _ map[mtime.Time]int, schedulable bool, pendingAdjustment int) { + minTs := mtime.MaxTimestamp + // TODO: Allow configurable limit of keys per bundle, and elements per key to improve parallelism. + // TODO: when we do, we need to ensure that the stage remains schedualable for bundle execution, for remaining pending elements and keys. + // With the greedy approach, we don't need to since "new data" triggers a refresh, and so should completing processing of a bundle. + newKeys := set[string]{} + + holdsInBundle := map[mtime.Time]int{} + + accumulatingPendingAdjustment := 0 +keysPerBundle: + for k, dnt := range ss.pendingByKeys { + if ss.inprogressKeys.present(k) { + continue + } + newKeys.insert(k) + // Track the min-timestamp for later watermark handling. + if dnt.elements[0].timestamp < minTs { + minTs = dnt.elements[0].timestamp + } + + var toProcessForKey []element + + // Can we pre-compute this bit when adding to pendingByKeys? + // startBundle is in run in a single scheduling goroutine, so moving per-element code + // to be computed by the bundle parallel goroutines will speed things up a touch. + for dnt.elements.Len() > 0 { + // For aggregations, only include elements for this key + // if we're after the end of window, or after the window expiry deadline. + // We will only ever trigger aggregations by watermark at most twice, once the watermark passes the window ends for OnTime completion, + // and once for when the window is closing. + elm := dnt.elements[0] + if watermark <= elm.window.MaxTimestamp() { + // The watermark hasn't passed the end of the window yet, we do nothing. + break + } + // Watermark is past the end of this window. Have we fired an OnTime pane yet? + state := ss.state[LinkID{}][elm.window][string(elm.keyBytes)] + // If this is not the ontime firing for this key. + + if state.Pane.Timing != typex.PaneEarly && watermark <= ss.strat.EarliestCompletion(elm.window) { + // The watermark is still before the earliest final completion for this window. + // Do not add further data for this firing. + // If this is the Never trigger, we also don't fire OnTime until after the earliest completion. + break + } + if ss.strat.IsNeverTrigger() && watermark <= ss.strat.EarliestCompletion(elm.window) { + // The NeverTrigger only has a single firing at the end of window + allowed lateness. + break + } + e := heap.Pop(&dnt.elements).(element) + + toProcessForKey = append(toProcessForKey, e) + } + // Get the pane for the aggregation correct, only mutate it once per key and window. + handledWindows := set[typex.Window]{} + for _, elm := range toProcessForKey { + state := ss.state[LinkID{}][elm.window][string(elm.keyBytes)] + if handledWindows.present(elm.window) { // The pane is already correct for this key + window + firing. if ss.strat.Accumulating && !state.Pane.IsLast { // If this isn't the last pane, then we must add the element back to the pending store for subsequent firings. heap.Push(&dnt.elements, elm) accumulatingPendingAdjustment++ } + continue + } + handledWindows.insert(elm.window) + + state.Pane = computeNextWatermarkPane(state.Pane) + // Determine if this is the last pane. + // Check if this is the post closing firing, which will be the last one. + // Unless it's the ontime pane, at which point it can never be last. + if watermark > ss.strat.EarliestCompletion(elm.window) && state.Pane.Timing != typex.PaneOnTime { + state.Pane.IsLast = true + } + if ss.strat.AllowedLateness == 0 || ss.strat.IsNeverTrigger() { + // If the allowed lateness is zero, then this will be the last pane. + // If this is the NeverTrigger, it's the last pane. + state.Pane.IsLast = true + } + ss.state[LinkID{}][elm.window][string(elm.keyBytes)] = state + + // The pane is already correct for this key + window + firing. + if ss.strat.Accumulating && !state.Pane.IsLast { + // If this isn't the last pane, then we must add the element back to the pending store for subsequent firings. + heap.Push(&dnt.elements, elm) + accumulatingPendingAdjustment++ } } + toProcess = append(toProcess, toProcessForKey...) if dnt.elements.Len() == 0 { @@ -1656,20 +1774,11 @@ keysPerBundle: break keysPerBundle } } - stillSchedulable := true - if ss.aggregate || (len(ss.pendingByKeys) == 0 && !timerCleared) { - // If this is an aggregate, we need a watermark change in order to reschedule - // If we're out of data, and timers were not cleared then the watermark is accurate. - stillSchedulable = false - } - if len(toProcess) == 0 { - // If we have nothing, there's nothing to progress. - return "", false, stillSchedulable, accumulatingPendingAdjustment - } + // If this is an aggregate, we need a watermark change in order to reschedule + stillSchedulable := false - bundID := ss.makeInProgressBundle(genBundID, toProcess, minTs, newKeys, holdsInBundle) - return bundID, true, stillSchedulable, accumulatingPendingAdjustment + return toProcess, minTs, newKeys, holdsInBundle, stillSchedulable, accumulatingPendingAdjustment } func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime.Time, genBundID func() string) (string, bool, bool) { @@ -1677,7 +1786,7 @@ func (ss *stageState) startProcessingTimeBundle(em *ElementManager, emNow mtime. defer ss.mu.Unlock() // TODO: Determine if it's possible and a good idea to treat all EventTime processing as a MinTime - // Special Case for ProcessintTime handling. + // Special Case for ProcessingTime handling. // Eg. Always queue EventTime elements at minTime. // Iterate all available processingTime events until we can't anymore. // @@ -1824,7 +1933,7 @@ func (ss *stageState) minPendingTimestampLocked() mtime.Time { func (ss *stageState) String() string { pcol, up := ss.UpstreamWatermark() - return fmt.Sprintf("[%v] IN: %v OUT: %v UP: %q %v, aggregation: %v", ss.ID, ss.input, ss.output, pcol, up, ss.aggregate) + return fmt.Sprintf("[%v] IN: %v OUT: %v UP: %q %v, kind: %v", ss.ID, ss.input, ss.output, pcol, up, ss.kind) } // updateWatermarks performs the following operations: From 29785cafa77dcf41a2a2a580e75655fbd7d6e924 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 19 Mar 2025 07:23:29 -0700 Subject: [PATCH 24/48] More precise binary operation inference. (#34305) * More precise binary operation inference. Handle numeric promotion, sequence multiplication, and fix division for Python 3.11+. * Fix older python versions. They have the opcode module, but no _nb_ops. * Restore fix for 3.9 and 3.10. * yapf --- sdks/python/apache_beam/typehints/opcodes.py | 40 +++++++++++++++---- .../typehints/trivial_inference_test.py | 16 ++++++++ 2 files changed, 48 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/typehints/opcodes.py b/sdks/python/apache_beam/typehints/opcodes.py index cd78e397f25c..e6fdcdba8e4d 100644 --- a/sdks/python/apache_beam/typehints/opcodes.py +++ b/sdks/python/apache_beam/typehints/opcodes.py @@ -54,6 +54,15 @@ # method on a C-implemented type will do. _MethodDescriptorType = type(str.upper) +if sys.version_info >= (3, 11): + import opcode + _div_binop_args = frozenset([ + ix for (ix, (argname, _)) in enumerate(opcode._nb_ops) + if 'TRUE_DIVIDE' in argname + ]) +else: + _div_binop_args = frozenset() + def pop_one(state, unused_arg): del state.stack[-1:] @@ -137,13 +146,32 @@ def get_iter(state, unused_arg): state.stack.append(Iterable[element_type(state.stack.pop())]) -def symmetric_binary_op(state, unused_arg): +_NUMERIC_PROMOTION_LADDER = [bool, int, float, complex] + + +def symmetric_binary_op(state, arg, is_true_div=None): # TODO(robertwb): This may not be entirely correct... b, a = Const.unwrap(state.stack.pop()), Const.unwrap(state.stack.pop()) if a == b: - state.stack.append(a) + if a is int and b is int and (arg in _div_binop_args or is_true_div): + state.stack.append(float) + else: + state.stack.append(a) elif type(a) == type(b) and isinstance(a, typehints.SequenceTypeConstraint): state.stack.append(type(a)(union(element_type(a), element_type(b)))) + # Technically these next two will be errors for anything but multiplication, + # but that's OK. + elif a is int and (b in (bytes, str) or + isinstance(b, typehints.SequenceTypeConstraint)): + state.stack.append(b) + elif b is int and (a in (bytes, str) or + isinstance(a, typehints.SequenceTypeConstraint)): + state.stack.append(a) + elif a in _NUMERIC_PROMOTION_LADDER and b in _NUMERIC_PROMOTION_LADDER: + state.stack.append( + _NUMERIC_PROMOTION_LADDER[max( + _NUMERIC_PROMOTION_LADDER.index(a), + _NUMERIC_PROMOTION_LADDER.index(b))]) else: state.stack.append(Any) @@ -155,12 +183,8 @@ def symmetric_binary_op(state, unused_arg): binary_floor_divide = inplace_floor_divide = symmetric_binary_op -def binary_true_divide(state, unused_arg): - u = union(state.stack.pop(), state.stack.pop) - if u == int: - state.stack.append(float) - else: - state.stack.append(u) +def binary_true_divide(state, arg): + return symmetric_binary_op(state, arg, True) inplace_true_divide = binary_true_divide diff --git a/sdks/python/apache_beam/typehints/trivial_inference_test.py b/sdks/python/apache_beam/typehints/trivial_inference_test.py index 9cea6d5794d8..c8b59c7ccbf7 100644 --- a/sdks/python/apache_beam/typehints/trivial_inference_test.py +++ b/sdks/python/apache_beam/typehints/trivial_inference_test.py @@ -244,6 +244,22 @@ def testBinOp(self): lambda a, b: a + b, [typehints.List[int], typehints.List[str]]) + def testBinOpPromotion(self): + self.assertReturnType(int, lambda a, b: a + b, [int, bool]) + self.assertReturnType(float, lambda a, b: a + b, [int, float]) + self.assertReturnType(complex, lambda a, b: a + b, [int, complex]) + + def testBinOpSequenceMul(self): + self.assertReturnType(str, lambda a, b: a * b, [int, str]) + self.assertReturnType(bytes, lambda a, b: a * b, [bytes, int]) + self.assertReturnType( + typehints.List[float], lambda a, b: a * b, [int, typehints.List[float]]) + + def testDiv(self): + # We only support Python 3 now. + self.assertReturnType(float, lambda a, b: a / b, [int, int]) + self.assertReturnType(int, lambda a, b: a // b, [int, int]) + def testCall(self): f = lambda x, *args: x self.assertReturnType( From d24cbf65e48c510424ea10e1f37dfc5350dd251b Mon Sep 17 00:00:00 2001 From: akashorabek <70029317+akashorabek@users.noreply.github.com> Date: Wed, 19 Mar 2025 19:33:04 +0500 Subject: [PATCH 25/48] Fix precommit flink container (#34250) * Fix Precommit Flink Container * Fix image tag in the cleanup step * Add beam-sdk-pr to public repos --- .../beam_PreCommit_Flink_Container.yml | 33 +++++++++++++++++-- .../python_Combine_Flink_Batch_small.txt | 1 - .../stale_dataflow_prebuilt_image_cleaner.sh | 2 +- .../beam/gradle/BeamModulePlugin.groovy | 2 +- 4 files changed, 32 insertions(+), 6 deletions(-) diff --git a/.github/workflows/beam_PreCommit_Flink_Container.yml b/.github/workflows/beam_PreCommit_Flink_Container.yml index d44f3a730928..e29ba9569914 100644 --- a/.github/workflows/beam_PreCommit_Flink_Container.yml +++ b/.github/workflows/beam_PreCommit_Flink_Container.yml @@ -77,6 +77,10 @@ env: HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.17_job_server:latest ARTIFACTS_DIR: gs://beam-flink-cluster/beam-precommit-flink-container-${{ github.run_id }} + DOCKER_REGISTRY: gcr.io + DOCKER_REPOSITORY_ROOT: ${{ github.event_name == 'pull_request_target' && 'gcr.io/apache-beam-testing/beam-sdk-pr' || 'gcr.io/apache-beam-testing/beam-sdk' }} + PYTHON_VERSION: 3.9 + PYTHON_SDK_IMAGE_TAG: latest jobs: beam_PreCommit_Flink_Container: @@ -87,7 +91,7 @@ jobs: github.event_name == 'pull_request_target' || github.event.comment.body == 'Run Flink Container PreCommit' runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 45 + timeout-minutes: 90 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: @@ -105,6 +109,24 @@ jobs: uses: ./.github/actions/setup-environment-action with: python-version: default + - name: GCloud Docker credential helper + if: ${{ github.event_name == 'pull_request_target' }} + run: | + gcloud auth configure-docker ${{ env.DOCKER_REGISTRY }} + - name: Set PYTHON_SDK_IMAGE_TAG unique variable based on timestamp + if: ${{ github.event_name == 'pull_request_target' }} + run: echo "PYTHON_SDK_IMAGE_TAG=$(date +'%Y%m%d-%H%M%S%N')" >> $GITHUB_ENV + - name: Build and push to registry + if: ${{ github.event_name == 'pull_request_target' }} + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:container:py39:docker + arguments: | + -PpythonVersion=${{ env.PYTHON_VERSION }} \ + -Pdocker-repository-root=${{ env.DOCKER_REPOSITORY_ROOT }} \ + -Pdocker-tag=${{ env.PYTHON_SDK_IMAGE_TAG }} \ + -PuseBuildx \ + -Ppush-containers - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: @@ -141,11 +163,11 @@ jobs: arguments: | -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ -Prunner=FlinkRunner \ - '-PloadTest.args=${{ env.beam_PreCommit_Flink_Container_test_arguments_2 }} --job_name=flink-tests-python-${{env.NOW_UTC}}' + '-PloadTest.args=${{ env.beam_PreCommit_Flink_Container_test_arguments_2 }} --environment_config=${{ env.DOCKER_REPOSITORY_ROOT }}/beam_python${{ env.PYTHON_VERSION }}_sdk:${{ env.PYTHON_SDK_IMAGE_TAG }} --job_name=flink-tests-python-${{env.NOW_UTC}}' # Run a Java Combine load test to verify the Flink container - name: Run Flink Container Test with Java Combine - timeout-minutes: 10 + timeout-minutes: 20 uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:java:testing:load-tests:run @@ -158,3 +180,8 @@ jobs: if: always() run: | ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete + + - name: Cleanup Python SDK Container + if: ${{ always() && github.event_name == 'pull_request_target' }} + run: | + gcloud container images delete ${{ env.DOCKER_REPOSITORY_ROOT }}/beam_python${{ env.PYTHON_VERSION }}_sdk:${{ env.PYTHON_SDK_IMAGE_TAG }} --force-delete-tags --quiet diff --git a/.github/workflows/flink-tests-pipeline-options/python_Combine_Flink_Batch_small.txt b/.github/workflows/flink-tests-pipeline-options/python_Combine_Flink_Batch_small.txt index 5522a8f9b823..8c2bd838e81b 100644 --- a/.github/workflows/flink-tests-pipeline-options/python_Combine_Flink_Batch_small.txt +++ b/.github/workflows/flink-tests-pipeline-options/python_Combine_Flink_Batch_small.txt @@ -18,6 +18,5 @@ --parallelism=2 --job_endpoint=localhost:8099 --environment_type=DOCKER ---environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.9_sdk:latest --top_count=10 --runner=PortableRunner \ No newline at end of file diff --git a/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh b/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh index 21181a9a192e..ed03ad6500ce 100755 --- a/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh +++ b/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh @@ -22,7 +22,7 @@ set -euo pipefail # Clean up private registry (us.gcr.io) # Images more than 5 day old and not the latest (either has latest label or newest) -PUBLIC_REPOSITORIES=(beam-sdk beam_portability beamgrafana beammetricssyncjenkins beammetricssyncgithub) +PUBLIC_REPOSITORIES=(beam-sdk beam-sdk-pr beam_portability beamgrafana beammetricssyncjenkins beammetricssyncgithub) PRIVATE_REPOSITORIES=(java-postcommit-it python-postcommit-it jenkins github-actions) # set as the same as 6-week release period if [[ $OSTYPE == "linux-gnu"* ]]; then 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 10135c842aa2..51ed48738b37 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -573,7 +573,7 @@ class BeamModulePlugin implements Plugin { } project.ext.useBuildx = { - return project.containerArchitectures() != [project.nativeArchitecture()] + return (project.containerArchitectures() != [project.nativeArchitecture()]) || project.rootProject.hasProperty("useBuildx") } /** ***********************************************************************************************/ From 04e97447c8cbc354dc8fe5d4da2a55745c0a84da Mon Sep 17 00:00:00 2001 From: So-shi Nakachi <40224073+Nakachi-S@users.noreply.github.com> Date: Wed, 19 Mar 2025 23:57:46 +0900 Subject: [PATCH 26/48] Aggregation option in Kinesis Writer Python sdk (#34323) * :+1: Add aggregation feature * :bug: Fix import bug * :white_check_mark: Add test * :+1: Add 3 argument * :bulb: Add comment * :art: Format code --- .../kinesis/KinesisTransformRegistrar.java | 36 ++++++++++++++++++- .../io/external/xlang_kinesisio_it_test.py | 20 +++++++++++ sdks/python/apache_beam/io/kinesis.py | 20 +++++++++++ 3 files changed, 75 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTransformRegistrar.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTransformRegistrar.java index 37f0ca09e024..5b4e094fe277 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTransformRegistrar.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTransformRegistrar.java @@ -64,6 +64,10 @@ private abstract static class CrossLanguageConfiguration { String region; @Nullable String serviceEndpoint; boolean verifyCertificate; + boolean aggregationEnabled; + Integer aggregationMaxBytes; + Duration aggregationMaxBufferedTime; + Duration aggregationShardRefreshInterval; public void setStreamName(String streamName) { this.streamName = streamName; @@ -88,6 +92,23 @@ public void setServiceEndpoint(@Nullable String serviceEndpoint) { public void setVerifyCertificate(@Nullable Boolean verifyCertificate) { this.verifyCertificate = verifyCertificate == null || verifyCertificate; } + + public void setAggregationEnabled(@Nullable Boolean aggregationEnabled) { + this.aggregationEnabled = aggregationEnabled != null && aggregationEnabled; + } + + public void setAggregationMaxBytes(Long aggregationMaxBytes) { + this.aggregationMaxBytes = aggregationMaxBytes.intValue(); + } + + public void setAggregationMaxBufferedTime(Long aggregationMaxBufferedTime) { + this.aggregationMaxBufferedTime = Duration.millis(aggregationMaxBufferedTime); + } + + public void setAggregationShardRefreshInterval(Long aggregationShardRefreshInterval) { + this.aggregationShardRefreshInterval = + Duration.standardMinutes(aggregationShardRefreshInterval); + } } public static class WriteBuilder @@ -131,9 +152,22 @@ public PTransform, KinesisIO.Write.Result> buildExternal( .skipCertificateVerification(!configuration.verifyCertificate) .build()) .withPartitioner(p -> pk) - .withRecordAggregationDisabled() .withSerializer(serializer); + if (configuration.aggregationEnabled) { + writeTransform = + writeTransform.withRecordAggregation( + KinesisIO.RecordAggregation.builder() + .maxBytes(configuration.aggregationMaxBytes) + .maxBufferedTimeJitter(0.7) // 70% jitter + .maxBufferedTime(configuration.aggregationMaxBufferedTime) + .shardRefreshIntervalJitter(0.5) // 50% jitter + .shardRefreshInterval(configuration.aggregationShardRefreshInterval) + .build()); + } else { + writeTransform = writeTransform.withRecordAggregationDisabled(); + } + return writeTransform; } } diff --git a/sdks/python/apache_beam/io/external/xlang_kinesisio_it_test.py b/sdks/python/apache_beam/io/external/xlang_kinesisio_it_test.py index c9181fb2a721..6a86248571e3 100644 --- a/sdks/python/apache_beam/io/external/xlang_kinesisio_it_test.py +++ b/sdks/python/apache_beam/io/external/xlang_kinesisio_it_test.py @@ -95,6 +95,7 @@ def test_kinesis_write(self): # TODO: remove this test once # https://github.com/apache/beam/issues/20416 is resolved self.run_kinesis_write() + self.run_kinesis_write_with_aggregation() records = self.kinesis_helper.read_from_stream(self.aws_kinesis_stream) self.assertEqual( sorted(records), @@ -118,6 +119,25 @@ def run_kinesis_write(self): verify_certificate=(not self.use_localstack), partition_key='1')) + def run_kinesis_write_with_aggregation(self): + with TestPipeline(options=PipelineOptions(self.pipeline_args)) as p: + p.not_use_test_runner_api = True + _ = ( + p + | 'Impulse' >> beam.Impulse() + | 'Generate' >> beam.FlatMap(lambda x: range(NUM_RECORDS)) # pylint: disable=bad-option-value + | 'Map to bytes' >> + beam.Map(lambda x: RECORD + str(x).encode()).with_output_types(bytes) + | 'WriteToKinesis' >> WriteToKinesis( + stream_name=self.aws_kinesis_stream, + aws_access_key=self.aws_access_key, + aws_secret_key=self.aws_secret_key, + region=self.aws_region, + service_endpoint=self.aws_service_endpoint, + verify_certificate=(not self.use_localstack), + partition_key='1', + aggregation_enabled=True)) + def run_kinesis_read(self): records = [RECORD + str(i).encode() for i in range(NUM_RECORDS)] diff --git a/sdks/python/apache_beam/io/kinesis.py b/sdks/python/apache_beam/io/kinesis.py index ce0bb2623a38..a066ec5cb2f0 100644 --- a/sdks/python/apache_beam/io/kinesis.py +++ b/sdks/python/apache_beam/io/kinesis.py @@ -112,6 +112,10 @@ def default_io_expansion_service(): ('partition_key', str), ('service_endpoint', Optional[str]), ('verify_certificate', Optional[bool]), + ('aggregation_enabled', Optional[bool]), + ('aggregation_max_bytes', Optional[int]), + ('aggregation_max_buffered_time', Optional[int]), + ('aggregation_shard_refresh_interval', Optional[int]), ], ) @@ -135,6 +139,10 @@ def __init__( verify_certificate=None, producer_properties=None, expansion_service=None, + aggregation_enabled=None, + aggregation_max_bytes=51200, + aggregation_max_buffered_time=100, + aggregation_shard_refresh_interval=2, ): """ Initializes a write operation to Kinesis. @@ -151,6 +159,13 @@ def __init__( since the AWS IOs upgraded to v2. Trying to set it will lead to an error. For more info, see https://github.com/apache/beam/issues/33430. :param expansion_service: The address (host:port) of the ExpansionService. + :param aggregation_enabled: Enable or disable aggregation. + :param aggregation_max_bytes: Maximum number of bytes to buffer before + sending a batch of records. Defaults to 51200. + :param aggregation_max_buffered_time: Maximum time(millisecond) to buffer + records before sending a batch of records. Defaults to 100. + :param aggregation_shard_refresh_interval: Interval in minutes to refresh + the shard map. Defaults to 2. """ if producer_properties is not None: raise ValueError( @@ -167,6 +182,11 @@ def __init__( partition_key=partition_key, service_endpoint=service_endpoint, verify_certificate=verify_certificate, + aggregation_enabled=aggregation_enabled, + aggregation_max_bytes=aggregation_max_bytes, + aggregation_max_buffered_time=aggregation_max_buffered_time, + aggregation_shard_refresh_interval= + aggregation_shard_refresh_interval, )), expansion_service or default_io_expansion_service(), ) From c6a8caa3ac19a9697bf4f26442d8c7008f65ef3a Mon Sep 17 00:00:00 2001 From: Rohit Sinha Date: Sat, 15 Mar 2025 18:07:44 -0700 Subject: [PATCH 27/48] Add an experiment named enableLineageRollup which when passed to Java SDK will make it report Lineage as BoundedTrie --- .../org/apache/beam/sdk/metrics/Lineage.java | 28 +++++++++++++++---- .../org/apache/beam/sdk/metrics/Metrics.java | 24 ++++++++++++++-- .../apache/beam/sdk/metrics/MetricsTest.java | 8 ++++++ 3 files changed, 51 insertions(+), 9 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java index b57f5f8bbf07..22928392038a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java @@ -24,6 +24,7 @@ import java.util.Set; import java.util.regex.Pattern; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.metrics.Metrics.MetricsFlag; 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.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -40,10 +41,17 @@ public class Lineage { // Reserved characters are backtick, colon, whitespace (space, \t, \n) and dot. private static final Pattern RESERVED_CHARS = Pattern.compile("[:\\s.`]"); - private final BoundedTrie metric; + private final Metric metric; private Lineage(Type type) { - this.metric = Metrics.boundedTrie(LINEAGE_NAMESPACE, type.toString()); + if (MetricsFlag.lineageRollupEnabled()) { + this.metric = + Metrics.boundedTrie( + LINEAGE_NAMESPACE, + type == Type.SOURCE ? Type.SOURCEV2.toString() : Type.SINKV2.toString()); + } else { + this.metric = Metrics.stringSet(LINEAGE_NAMESPACE, type.toString()); + } } /** {@link Lineage} representing sources and optionally side inputs. */ @@ -92,7 +100,6 @@ static Iterable getFQNParts( } } } - return parts; } @@ -134,7 +141,11 @@ public void add(String system, Iterable segments) { */ public void add(Iterable rollupSegments) { ImmutableList segments = ImmutableList.copyOf(rollupSegments); - this.metric.add(segments); + if (MetricsFlag.lineageRollupEnabled()) { + ((BoundedTrie) this.metric).add(segments); + } else { + ((StringSet) this.metric).add(String.join("", segments)); + } } /** @@ -183,8 +194,13 @@ public static Set query(MetricResults results, Type type) { /** Lineage metrics resource types. */ public enum Type { - SOURCE("sources_v2"), - SINK("sinks_v2"); + // Used by StringSet to report lineage metrics + SOURCE("sources"), + SINK("sinks"), + + // Used by BoundedTrie to report lineage metrics + SOURCEV2("sources_v2"), + SINKV2("sinks_v2"); private final String name; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java index 7af782099c20..b9f58d3b9ff1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java @@ -66,12 +66,17 @@ static class MetricsFlag { final boolean counterDisabled; final boolean stringSetDisabled; final boolean boundedTrieDisabled; + final boolean lineageRollupEnabled; private MetricsFlag( - boolean counterDisabled, boolean stringSetDisabled, boolean boundedTrieDisabled) { + boolean counterDisabled, + boolean stringSetDisabled, + boolean boundedTrieDisabled, + boolean lineageRollupEnabled) { this.counterDisabled = counterDisabled; this.stringSetDisabled = stringSetDisabled; this.boundedTrieDisabled = boundedTrieDisabled; + this.lineageRollupEnabled = lineageRollupEnabled; } static boolean counterDisabled() { @@ -88,6 +93,11 @@ static boolean boundedTrieDisabled() { MetricsFlag flag = INSTANCE.get(); return flag != null && flag.boundedTrieDisabled; } + + static boolean lineageRollupEnabled() { + MetricsFlag flag = INSTANCE.get(); + return flag != null && flag.lineageRollupEnabled; + } } /** @@ -114,8 +124,14 @@ public static void setDefaultPipelineOptions(PipelineOptions options) { if (boundedTrieDisabled) { LOG.info("BoundedTrie metrics are disabled"); } + boolean lineageRollupEnabled = ExperimentalOptions.hasExperiment(exp, "enableLineageRollup"); + if (lineageRollupEnabled) { + LOG.info("Lineage Rollup is enabled. Will use BoundedTrie to track lineage."); + } MetricsFlag.INSTANCE.compareAndSet( - null, new MetricsFlag(counterDisabled, stringSetDisabled, boundedTrieDisabled)); + null, + new MetricsFlag( + counterDisabled, stringSetDisabled, boundedTrieDisabled, lineageRollupEnabled)); } } @@ -309,7 +325,9 @@ public MetricName getName() { @Override public void add(Iterable values) { - if (MetricsFlag.boundedTrieDisabled()) { + // If lineageRollupEnabled is not set explicitly then lineage is + // not emitted as BounedTrie. + if (MetricsFlag.boundedTrieDisabled() || !MetricsFlag.lineageRollupEnabled()) { return; } MetricsContainer container = MetricsEnvironment.getCurrentContainer(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java index 5a278858bd4e..2fc800adca56 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java @@ -264,6 +264,7 @@ public void testMetricsFlag() { assertFalse(Metrics.MetricsFlag.counterDisabled()); assertFalse(Metrics.MetricsFlag.stringSetDisabled()); assertFalse(Metrics.MetricsFlag.boundedTrieDisabled()); + assertFalse(Metrics.MetricsFlag.lineageRollupEnabled()); PipelineOptions options = PipelineOptionsFactory.fromArgs("--experiments=disableCounterMetrics").create(); Metrics.setDefaultPipelineOptions(options); @@ -281,6 +282,13 @@ public void testMetricsFlag() { assertFalse(Metrics.MetricsFlag.stringSetDisabled()); assertTrue(Metrics.MetricsFlag.boundedTrieDisabled()); Metrics.resetDefaultPipelineOptions(); + options = PipelineOptionsFactory.fromArgs("--experiments=enableLineageRollup").create(); + Metrics.setDefaultPipelineOptions(options); + assertFalse(Metrics.MetricsFlag.counterDisabled()); + assertFalse(Metrics.MetricsFlag.stringSetDisabled()); + assertFalse(Metrics.MetricsFlag.boundedTrieDisabled()); + assertTrue(Metrics.MetricsFlag.lineageRollupEnabled()); + Metrics.resetDefaultPipelineOptions(); } } From 0af1ede89cd8d512d124f4b8a53570e508a53484 Mon Sep 17 00:00:00 2001 From: Rohit Sinha Date: Mon, 17 Mar 2025 10:46:09 -0700 Subject: [PATCH 28/48] Allow boundedtrie emission outside of lineage usecase --- .../src/main/java/org/apache/beam/sdk/metrics/Metrics.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java index b9f58d3b9ff1..d18e3ee96d69 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java @@ -325,9 +325,8 @@ public MetricName getName() { @Override public void add(Iterable values) { - // If lineageRollupEnabled is not set explicitly then lineage is - // not emitted as BounedTrie. - if (MetricsFlag.boundedTrieDisabled() || !MetricsFlag.lineageRollupEnabled()) { + // If BoundedTrie metrics are disabled explicitly then do not emit them. + if (MetricsFlag.boundedTrieDisabled()) { return; } MetricsContainer container = MetricsEnvironment.getCurrentContainer(); From b2372b2fa4c987bcf9d6c3aeebd514ea2d47357a Mon Sep 17 00:00:00 2001 From: Rohit Sinha Date: Tue, 18 Mar 2025 10:22:14 -0700 Subject: [PATCH 29/48] Update Lineage query result to support BoundedTrie and StringSet both --- .../org/apache/beam/sdk/metrics/Lineage.java | 47 ++++++++++++++++--- 1 file changed, 41 insertions(+), 6 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java index 22928392038a..1e0124fc518b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java @@ -158,13 +158,10 @@ public void add(Iterable rollupSegments) { * truncatedMarker. */ public static Set query(MetricResults results, Type type, String truncatedMarker) { - MetricsFilter filter = - MetricsFilter.builder() - .addNameFilter(MetricNameFilter.named(LINEAGE_NAMESPACE, type.toString())) - .build(); + MetricQueryResults lineageQueryResults = getLineageQueryResults(results, type); Set result = new HashSet<>(); truncatedMarker = truncatedMarker == null ? "*" : truncatedMarker; - for (MetricResult metrics : results.queryMetrics(filter).getBoundedTries()) { + for (MetricResult metrics : lineageQueryResults.getBoundedTries()) { try { for (List fqn : metrics.getCommitted().getResult()) { String end = Boolean.parseBoolean(fqn.get(fqn.size() - 1)) ? truncatedMarker : ""; @@ -189,7 +186,18 @@ public static Set query(MetricResults results, Type type, String truncat * @return A flat representation of all FQNs. If the FQN was truncated then it has a trailing '*'. */ public static Set query(MetricResults results, Type type) { - return query(results, type, "*"); + if (MetricsFlag.lineageRollupEnabled()) { + // If user accidentally end up specifying V1 type then override it with V2. + if (type == Type.SOURCE) { + type = Type.SOURCEV2; + } + if (type == Type.SINK) { + type = Type.SINKV2; + } + return query(results, type, "*"); + } else { + return queryLineageV1(results, type); + } } /** Lineage metrics resource types. */ @@ -214,6 +222,33 @@ public String toString() { } } + /** + * Query {@link StringSet} metrics from {@link MetricResults}. This method is kept as-is from + * previous Beam release for removal once BoundedTrie is made default. + */ + private static Set queryLineageV1(MetricResults results, Type type) { + MetricQueryResults lineageQueryResults = getLineageQueryResults(results, type); + Set result = new HashSet<>(); + for (MetricResult metrics : lineageQueryResults.getStringSets()) { + try { + result.addAll(metrics.getCommitted().getStringSet()); + } catch (UnsupportedOperationException unused) { + // MetricsResult.getCommitted throws this exception when runner support missing, just skip. + } + result.addAll(metrics.getAttempted().getStringSet()); + } + return result; + } + + /** @return {@link MetricQueryResults} containing lineage metrics. */ + private static MetricQueryResults getLineageQueryResults(MetricResults results, Type type) { + MetricsFilter filter = + MetricsFilter.builder() + .addNameFilter(MetricNameFilter.named(LINEAGE_NAMESPACE, type.toString())) + .build(); + return results.queryMetrics(filter); + } + /** * Wrap segment to valid segment name. * From 1f68c4950fec2c847b4bd09a5d4e0e699b10e70f Mon Sep 17 00:00:00 2001 From: scwhittle Date: Wed, 19 Mar 2025 17:34:56 +0100 Subject: [PATCH 30/48] update CHANGES with Read bugs/improvements (#34344) --- CHANGES.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 261cab20719f..91c6edba3502 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -92,7 +92,9 @@ * (Java) Fixed TIME field encodings for BigQuery Storage API writes on GenericRecords ([#34059](https://github.com/apache/beam/pull/34059)). * (Java) Fixed a race condition in JdbcIO which could cause hangs trying to acquire a connection ([#34058](https://github.com/apache/beam/pull/34058)). * (Java) Fix BigQuery Storage Write compatibility with Avro 1.8 ([#34281](https://github.com/apache/beam/pull/34281)). -* Fixed checkpoint recovery and streaming behavior in Spark Classic and Portable runner's Flatten transform by replacing queueStream with SingleEmitInputDStream ([#34080](https://github.com/apache/beam/pull/34080), [#18144](https://github.com/apache/beam/issues/18144), [#20426](https://github.com/apache/beam/issues/20426)). +* Fixed checkpoint recovery and streaming behavior in Spark Classic and Portable runner's Flatten transform by replacing queueStream with SingleEmitInputDStream ([#34080](https://github.com/apache/beam/pull/34080), [#18144](https://github.com/apache/beam/issues/18144), [#20426](https://github.com/apache/beam/issues/20426)) +* (Java) Fixed Read caching of UnboundedReader objects to effectively cache across multiple DoFns and avoid checkpointing unstarted reader. [#34146](https://github.com/apache/beam/pull/34146) [#33901](https://github.com/apache/beam/pull/33901) + ## Security Fixes * Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). From 6c557fe6b907a5a390e66e1ce699eddc43026ff3 Mon Sep 17 00:00:00 2001 From: Rohit Sinha Date: Thu, 13 Mar 2025 22:02:50 -0700 Subject: [PATCH 31/48] Revert Skip BoundedTrie on Dataflow till service is have BoundedTrie #33921 --- .../google-cloud-dataflow-java/build.gradle | 2 -- .../worker/BatchModeExecutionContext.java | 19 ++++--------------- .../worker/StreamingStepMetricsContainer.java | 6 +----- .../StreamingStepMetricsContainerTest.java | 3 --- 4 files changed, 5 insertions(+), 25 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 11a2d333cfa0..415f11455d62 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -193,7 +193,6 @@ def commonLegacyExcludeCategories = [ 'org.apache.beam.sdk.testing.UsesParDoLifecycle', // doesn't support remote runner 'org.apache.beam.sdk.testing.UsesMetricsPusher', 'org.apache.beam.sdk.testing.UsesBundleFinalizer', - 'org.apache.beam.sdk.testing.UsesBoundedTrieMetrics', ] def commonRunnerV2ExcludeCategories = [ @@ -207,7 +206,6 @@ def commonRunnerV2ExcludeCategories = [ 'org.apache.beam.sdk.testing.UsesTestStream', 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime', 'org.apache.beam.sdk.testing.UsesRequiresTimeSortedInput', - 'org.apache.beam.sdk.testing.UsesBoundedTrieMetrics', ] // For the following test tasks using legacy worker, set workerHarnessContainerImage to empty to diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java index 2d8638b3330f..1bfaaceb8253 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java @@ -19,7 +19,6 @@ import com.google.api.services.dataflow.model.CounterUpdate; import com.google.api.services.dataflow.model.SideInputInfo; -import java.util.Collections; import java.util.Objects; import java.util.concurrent.TimeUnit; import org.apache.beam.runners.core.InMemoryStateInternals; @@ -78,9 +77,6 @@ public class BatchModeExecutionContext protected static final String BIGQUERY_READ_THROTTLE_TIME_NAMESPACE = "org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$StorageClientImpl"; - // TODO(BEAM-33720): Remove once Dataflow legacy runner supports BoundedTries. - private final boolean populateBoundedTrieMetrics; - private BatchModeExecutionContext( CounterFactory counterFactory, Cache> dataCache, @@ -88,8 +84,7 @@ private BatchModeExecutionContext( ReaderFactory readerFactory, PipelineOptions options, DataflowExecutionStateTracker executionStateTracker, - DataflowExecutionStateRegistry executionStateRegistry, - boolean populateBoundedTrieMetrics) { + DataflowExecutionStateRegistry executionStateRegistry) { super( counterFactory, createMetricsContainerRegistry(), @@ -102,7 +97,6 @@ private BatchModeExecutionContext( this.dataCache = dataCache; this.containerRegistry = (MetricsContainerRegistry) getMetricsContainerRegistry(); - this.populateBoundedTrieMetrics = populateBoundedTrieMetrics; } private static MetricsContainerRegistry createMetricsContainerRegistry() { @@ -138,8 +132,7 @@ public static BatchModeExecutionContext forTesting( counterFactory, options, "test-work-item-id"), - stateRegistry, - true); + stateRegistry); } public static BatchModeExecutionContext forTesting(PipelineOptions options, String stageName) { @@ -252,8 +245,7 @@ public static BatchModeExecutionContext create( counterFactory, options, workItemId), - executionStateRegistry, - false); + executionStateRegistry); } /** Create a new {@link StepContext}. */ @@ -528,10 +520,7 @@ public Iterable extractMetricUpdates(boolean isFinalUpdate) { update -> MetricsToCounterUpdateConverter.fromStringSet( update.getKey(), true, update.getUpdate())), - FluentIterable.from( - populateBoundedTrieMetrics - ? updates.boundedTrieUpdates() - : Collections.emptyList()) + FluentIterable.from(updates.boundedTrieUpdates()) .transform( update -> MetricsToCounterUpdateConverter.fromBoundedTrie( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java index ca7df40762f4..1a3594a973f3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java @@ -23,7 +23,6 @@ import java.time.Clock; import java.time.Duration; import java.time.Instant; -import java.util.Collections; import java.util.HashSet; import java.util.Map; import java.util.Map.Entry; @@ -104,9 +103,6 @@ public class StreamingStepMetricsContainer implements MetricsContainer { private final Clock clock; - // TODO(BEAM-33720): Remove once Dataflow legacy runner supports BoundedTries. - @VisibleForTesting boolean populateBoundedTrieMetrics; - private StreamingStepMetricsContainer(String stepName) { this.stepName = stepName; this.perWorkerCountersByFirstStaleTime = new ConcurrentHashMap<>(); @@ -217,7 +213,7 @@ public Iterable extractUpdates() { .append(distributionUpdates()) .append(gaugeUpdates()) .append(stringSetUpdates()) - .append(populateBoundedTrieMetrics ? boundedTrieUpdates() : Collections.emptyList()); + .append(boundedTrieUpdates()); } private FluentIterable counterUpdates() { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java index 48d8d2dbf4a1..02ce0b2f8706 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java @@ -371,7 +371,6 @@ public void testBoundedTrieUpdateExtraction() { .setBoundedTrie( MetricsToCounterUpdateConverter.getBoundedTrie(expectedName1.toProto())); - ((StreamingStepMetricsContainer) c1).populateBoundedTrieMetrics = true; Iterable updates = StreamingStepMetricsContainer.extractMetricUpdates(registry); assertThat(updates, containsInAnyOrder(name1Update)); @@ -400,7 +399,6 @@ public void testBoundedTrieUpdateExtraction() { .setBoundedTrie( MetricsToCounterUpdateConverter.getBoundedTrie(expectedName2.toProto())); - ((StreamingStepMetricsContainer) c2).populateBoundedTrieMetrics = true; updates = StreamingStepMetricsContainer.extractMetricUpdates(registry); assertThat(updates, containsInAnyOrder(name2Update)); @@ -412,7 +410,6 @@ public void testBoundedTrieUpdateExtraction() { name1Update.setBoundedTrie( MetricsToCounterUpdateConverter.getBoundedTrie(expectedName1.toProto())); - ((StreamingStepMetricsContainer) c1).populateBoundedTrieMetrics = true; updates = StreamingStepMetricsContainer.extractMetricUpdates(registry); assertThat(updates, containsInAnyOrder(name1Update)); } From 2710ba8e763458a21d6d664476a305bf05d9f9be Mon Sep 17 00:00:00 2001 From: Rohit Sinha Date: Tue, 18 Mar 2025 22:11:40 -0700 Subject: [PATCH 32/48] Do not runner UsesBoundedTrie test on Dataflow runner as the test verify metric result --- .../beam_PostCommit_Java_ValidatesRunner_Dataflow.json | 3 ++- runners/google-cloud-dataflow-java/build.gradle | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json index e3d6056a5de9..8aad4bae0703 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2, + "https://github.com/apache/beam/pull/34294": "noting that PR #34294 should run this test" } diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 415f11455d62..4c25737fe2f7 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -193,6 +193,7 @@ def commonLegacyExcludeCategories = [ 'org.apache.beam.sdk.testing.UsesParDoLifecycle', // doesn't support remote runner 'org.apache.beam.sdk.testing.UsesMetricsPusher', 'org.apache.beam.sdk.testing.UsesBundleFinalizer', + 'org.apache.beam.sdk.testing.UsesBoundedTrieMetrics', // Dataflow QM as of now does not support returning back BoundedTrie in metric result. ] def commonRunnerV2ExcludeCategories = [ @@ -206,6 +207,7 @@ def commonRunnerV2ExcludeCategories = [ 'org.apache.beam.sdk.testing.UsesTestStream', 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime', 'org.apache.beam.sdk.testing.UsesRequiresTimeSortedInput', + 'org.apache.beam.sdk.testing.UsesBoundedTrieMetrics', // Dataflow QM as of now does not support returning back BoundedTrie in metric result. ] // For the following test tasks using legacy worker, set workerHarnessContainerImage to empty to From 00cb04b944ba900b9b76dc80c4066558996bf36d Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 19 Mar 2025 13:41:26 -0400 Subject: [PATCH 33/48] [#31438] Add changes.md message for Trigger support in Prism. (#34346) * [#31438] Add changes.md message. * include missing word "support" --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- CHANGES.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 91c6edba3502..5e19cfbe617e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -74,6 +74,8 @@ * [Java] Support for `--add-modules` JVM option is added through a new pipeline option `JdkAddRootModules`. This allows extending the module graph with optional modules such as SDK incubator modules. Sample usage: ` --jdkAddRootModules=jdk.incubator.vector` ([#30281](https://github.com/apache/beam/issues/30281)). * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Managed API for [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/managed/Managed.html) and [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.managed.html#module-apache_beam.transforms.managed) supports [key I/O connectors](https://beam.apache.org/documentation/io/connectors/) Iceberg, Kafka, and BigQuery. +* Prism now supports event time triggers for most common cases. ([#31438](https://github.com/apache/beam/issues/31438)) + * Prism does not yet support triggered side inputs, or triggers on merging windows (such as session windows). ## Breaking Changes From a437dc4f2da4e98632486a29de65fa09a45c5f2b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 19 Mar 2025 10:43:17 -0700 Subject: [PATCH 34/48] Bump cloud.google.com/go/storage from 1.50.0 to 1.51.0 in /sdks (#34340) Bumps [cloud.google.com/go/storage](https://github.com/googleapis/google-cloud-go) from 1.50.0 to 1.51.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.50.0...spanner/v1.51.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 | 40 ++++++++++++------------ sdks/go.sum | 87 +++++++++++++++++++++++++++-------------------------- 2 files changed, 64 insertions(+), 63 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index ce2c9034100c..013d60904913 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,8 +30,8 @@ require ( cloud.google.com/go/datastore v1.20.0 cloud.google.com/go/profiler v0.4.2 cloud.google.com/go/pubsub v1.47.0 - cloud.google.com/go/spanner v1.75.0 - cloud.google.com/go/storage v1.50.0 + cloud.google.com/go/spanner v1.76.1 + cloud.google.com/go/storage v1.51.0 github.com/aws/aws-sdk-go-v2 v1.36.3 github.com/aws/aws-sdk-go-v2/config v1.29.6 github.com/aws/aws-sdk-go-v2/credentials v1.17.59 @@ -41,7 +41,7 @@ require ( github.com/docker/go-connections v0.5.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.9.0 - github.com/google/go-cmp v0.6.0 + github.com/google/go-cmp v0.7.0 github.com/google/uuid v1.6.0 github.com/johannesboyne/gofakes3 v0.0.0-20250106100439-5c39aecd6999 github.com/lib/pq v1.10.9 @@ -60,9 +60,9 @@ require ( golang.org/x/sync v0.12.0 golang.org/x/sys v0.31.0 golang.org/x/text v0.23.0 - google.golang.org/api v0.221.0 - google.golang.org/genproto v0.0.0-20250122153221-138b5a5a4fd4 - google.golang.org/grpc v1.70.0 + google.golang.org/api v0.224.0 + google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb + google.golang.org/grpc v1.71.0 google.golang.org/protobuf v1.36.5 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 @@ -77,16 +77,16 @@ require ( require ( cel.dev/expr v0.19.2 // indirect - cloud.google.com/go/auth v0.14.1 // indirect + cloud.google.com/go/auth v0.15.0 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.7 // indirect - cloud.google.com/go/monitoring v1.23.0 // indirect + cloud.google.com/go/monitoring v1.24.0 // indirect dario.cat/mergo v1.0.1 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/AdaLogics/go-fuzz-headers v0.0.0-20240806141605-e8a1dd7889d6 // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.2 // indirect github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.49.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.49.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.51.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.51.0 // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect github.com/containerd/log v0.1.0 // indirect github.com/containerd/platforms v0.2.1 // indirect @@ -115,24 +115,24 @@ require ( github.com/yusufpapurcu/wmi v1.2.4 // indirect go.einride.tech/aip v0.68.1 // indirect go.opentelemetry.io/auto/sdk v1.1.0 // indirect - go.opentelemetry.io/contrib/detectors/gcp v1.33.0 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.58.0 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.58.0 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.34.0 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.59.0 // indirect go.opentelemetry.io/otel v1.34.0 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.33.0 // indirect go.opentelemetry.io/otel/metric v1.34.0 // indirect go.opentelemetry.io/otel/sdk v1.34.0 // indirect - go.opentelemetry.io/otel/sdk/metric v1.33.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.34.0 // indirect go.opentelemetry.io/otel/trace v1.34.0 // indirect go.shabbyrobe.org/gocovmerge v0.0.0-20230507111327-fa4f82cfbf4d // indirect golang.org/x/time v0.10.0 // indirect ) require ( - cloud.google.com/go v0.118.1 // indirect + cloud.google.com/go v0.118.3 // indirect cloud.google.com/go/compute/metadata v0.6.0 // indirect - cloud.google.com/go/iam v1.3.1 // indirect - cloud.google.com/go/longrunning v0.6.4 // indirect + cloud.google.com/go/iam v1.4.1 // indirect + cloud.google.com/go/longrunning v0.6.5 // indirect github.com/Azure/go-ansiterm v0.0.0-20250102033503-faa5f7b0171c // indirect github.com/Microsoft/go-winio v0.6.2 // indirect github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40 // indirect @@ -167,7 +167,7 @@ require ( github.com/google/pprof v0.0.0-20241210010833-40e02aabc2ad // indirect github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.9 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.3.4 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.5 // indirect github.com/googleapis/gax-go/v2 v2.14.1 // indirect github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect @@ -198,8 +198,8 @@ require ( golang.org/x/mod v0.22.0 // indirect golang.org/x/tools v0.29.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20250127172529-29210b9bc287 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20250303144028-a0af3efb3deb // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250303144028-a0af3efb3deb // indirect ) // Hopefully a temporary measure to sort out a mismatch between grpc packages diff --git a/sdks/go.sum b/sdks/go.sum index b36a229e24ee..509d325e71a6 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -40,8 +40,8 @@ cloud.google.com/go v0.104.0/go.mod h1:OO6xxXdJyvuJPcEPBLN9BJPD+jep5G1+2U5B5gkRY cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= cloud.google.com/go v0.107.0/go.mod h1:wpc2eNrD7hXUTy8EKS10jkxpZBjASrORK7goS+3YX2I= cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= -cloud.google.com/go v0.118.1 h1:b8RATMcrK9A4BH0rj8yQupPXp+aP+cJ0l6H7V9osV1E= -cloud.google.com/go v0.118.1/go.mod h1:CFO4UPEPi8oV21xoezZCrd3d81K4fFkDTEJu4R8K+9M= +cloud.google.com/go v0.118.3 h1:jsypSnrE/w4mJysioGdMBg4MiW/hHx/sArFpaBWHdME= +cloud.google.com/go v0.118.3/go.mod h1:Lhs3YLnBlwJ4KA6nuObNMZ/fCbOQBPuWKPoE0Wa/9Vc= cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= @@ -103,8 +103,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.14.1 h1:AwoJbzUdxA/whv1qj3TLKwh3XX5sikny2fc40wUl+h0= -cloud.google.com/go/auth v0.14.1/go.mod h1:4JHUxlGXisL0AW8kXPtUF6ztuOksyfUQNFjfsOCXkPM= +cloud.google.com/go/auth v0.15.0 h1:Ly0u4aA5vG/fsSsxu98qCQBemXtAtJf+95z9HK+cxps= +cloud.google.com/go/auth v0.15.0/go.mod h1:WJDGqZ1o9E9wKIL+IwStfyn/+s59zl4Bi+1KQNVXLZ8= cloud.google.com/go/auth/oauth2adapt v0.2.7 h1:/Lc7xODdqcEw8IrZ9SvwnlLX6j9FHQM74z6cBk9Rw6M= cloud.google.com/go/auth/oauth2adapt v0.2.7/go.mod h1:NTbTTzfvPl1Y3V1nPpOgl2w6d/FjO7NNUQaWSox6ZMc= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= @@ -332,8 +332,8 @@ cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGE cloud.google.com/go/iam v0.11.0/go.mod h1:9PiLDanza5D+oWFZiH1uG+RnRCfEGKoyl6yo4cgWZGY= cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= cloud.google.com/go/iam v0.13.0/go.mod h1:ljOg+rcNfzZ5d6f1nAUJ8ZIxOaZUVoS14bKCtaLZ/D0= -cloud.google.com/go/iam v1.3.1 h1:KFf8SaT71yYq+sQtRISn90Gyhyf4X8RGgeAVC8XGf3E= -cloud.google.com/go/iam v1.3.1/go.mod h1:3wMtuyT4NcbnYNPLMBzYRFiEfjKfJlLVLrisE7bwm34= +cloud.google.com/go/iam v1.4.1 h1:cFC25Nv+u5BkTR/BT1tXdoF2daiVbZ1RLx2eqfQ9RMM= +cloud.google.com/go/iam v1.4.1/go.mod h1:2vUEJpUG3Q9p2UdsyksaKpDzlwOrnMzS30isdReIcLM= cloud.google.com/go/iap v1.4.0/go.mod h1:RGFwRJdihTINIe4wZ2iCP0zF/qu18ZwyKxrhMhygBEc= cloud.google.com/go/iap v1.5.0/go.mod h1:UH/CGgKd4KyohZL5Pt0jSKE4m3FR51qg6FKQ/z/Ix9A= cloud.google.com/go/iap v1.6.0/go.mod h1:NSuvI9C/j7UdjGjIde7t7HBz+QTwBcapPE07+sSRcLk= @@ -354,8 +354,8 @@ cloud.google.com/go/kms v1.8.0/go.mod h1:4xFEhYFqvW+4VMELtZyxomGSYtSQKzM178ylFW4 cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8dh3w= cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= -cloud.google.com/go/kms v1.20.5 h1:aQQ8esAIVZ1atdJRxihhdxGQ64/zEbJoJnCz/ydSmKg= -cloud.google.com/go/kms v1.20.5/go.mod h1:C5A8M1sv2YWYy1AE6iSrnddSG9lRGdJq5XEdBy28Lmw= +cloud.google.com/go/kms v1.21.0 h1:x3EeWKuYwdlo2HLse/876ZrKjk2L5r7Uexfm8+p6mSI= +cloud.google.com/go/kms v1.21.0/go.mod h1:zoFXMhVVK7lQ3JC9xmhHMoQhnjEDZFoLAr5YMwzBLtk= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -371,8 +371,8 @@ cloud.google.com/go/logging v1.13.0/go.mod h1:36CoKh6KA/M0PbhPKMq6/qety2DCAErbhX cloud.google.com/go/longrunning v0.1.1/go.mod h1:UUFxuDWkv22EuY93jjmDMFT5GPQKeFVJBIF6QlTqdsE= cloud.google.com/go/longrunning v0.3.0/go.mod h1:qth9Y41RRSUE69rDcOn6DdK3HfQfsUI0YSmW3iIlLJc= cloud.google.com/go/longrunning v0.4.1/go.mod h1:4iWDqhBZ70CvZ6BfETbvam3T8FMvLK+eFj0E6AaRQTo= -cloud.google.com/go/longrunning v0.6.4 h1:3tyw9rO3E2XVXzSApn1gyEEnH2K9SynNQjMlBi3uHLg= -cloud.google.com/go/longrunning v0.6.4/go.mod h1:ttZpLCe6e7EXvn9OxpBRx7kZEB0efv8yBO6YnVMfhJs= +cloud.google.com/go/longrunning v0.6.5 h1:sD+t8DO8j4HKW4QfouCklg7ZC1qC4uzVZt8iz3uTW+Q= +cloud.google.com/go/longrunning v0.6.5/go.mod h1:Et04XK+0TTLKa5IPYryKf5DkpwImy6TluQ1QTLwlKmY= cloud.google.com/go/managedidentities v1.3.0/go.mod h1:UzlW3cBOiPrzucO5qWkNkh0w33KFtBJU281hacNvsdE= cloud.google.com/go/managedidentities v1.4.0/go.mod h1:NWSBYbEMgqmbZsLIyKvxrYbtqOsxY1ZrGM+9RgDqInM= cloud.google.com/go/managedidentities v1.5.0/go.mod h1:+dWcZ0JlUmpuxpIDfyP5pP5y0bLdRwOS4Lp7gMni/LA= @@ -398,8 +398,8 @@ cloud.google.com/go/monitoring v1.7.0/go.mod h1:HpYse6kkGo//7p6sT0wsIC6IBDET0RhI cloud.google.com/go/monitoring v1.8.0/go.mod h1:E7PtoMJ1kQXWxPjB6mv2fhC5/15jInuulFdYYtlcvT4= cloud.google.com/go/monitoring v1.12.0/go.mod h1:yx8Jj2fZNEkL/GYZyTLS4ZtZEZN8WtDEiEqG4kLK50w= cloud.google.com/go/monitoring v1.13.0/go.mod h1:k2yMBAB1H9JT/QETjNkgdCGD9bPF712XiLTVr+cBrpw= -cloud.google.com/go/monitoring v1.23.0 h1:M3nXww2gn9oZ/qWN2bZ35CjolnVHM3qnSbu6srCPgjk= -cloud.google.com/go/monitoring v1.23.0/go.mod h1:034NnlQPDzrQ64G2Gavhl0LUHZs9H3rRmhtnp7jiJgg= +cloud.google.com/go/monitoring v1.24.0 h1:csSKiCJ+WVRgNkRzzz3BPoGjFhjPY23ZTcaenToJxMM= +cloud.google.com/go/monitoring v1.24.0/go.mod h1:Bd1PRK5bmQBQNnuGwHBfUamAV1ys9049oEPHnn4pcsc= cloud.google.com/go/networkconnectivity v1.4.0/go.mod h1:nOl7YL8odKyAOtzNX73/M5/mGZgqqMeryi6UPZTk/rA= cloud.google.com/go/networkconnectivity v1.5.0/go.mod h1:3GzqJx7uhtlM3kln0+x5wyFvuVH1pIBJjhCpjzSt75o= cloud.google.com/go/networkconnectivity v1.6.0/go.mod h1:OJOoEXW+0LAxHh89nXd64uGG+FbQoeH8DtxCHVOMlaM= @@ -552,8 +552,8 @@ cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+ cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= -cloud.google.com/go/spanner v1.75.0 h1:2zrltTJv/4P3pCgpYgde4Eb1vN8Cgy1fNy7pbTnOovg= -cloud.google.com/go/spanner v1.75.0/go.mod h1:TLFZBvPQmx3We7sGh12eTk9lLsRLczzZaiweqfMpR80= +cloud.google.com/go/spanner v1.76.1 h1:vYbVZuXfnFwvNcvH3lhI2PeUA+kHyqKmLC7mJWaC4Ok= +cloud.google.com/go/spanner v1.76.1/go.mod h1:YtwoE+zObKY7+ZeDCBtZ2ukM+1/iPaMfUM+KnTh/sx0= cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= @@ -573,8 +573,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.50.0 h1:3TbVkzTooBvnZsk7WaAQfOsNrdoM8QHusXA1cpk6QJs= -cloud.google.com/go/storage v1.50.0/go.mod h1:l7XeiD//vx5lfqE3RavfmU9yvk5Pp0Zhcv482poyafY= +cloud.google.com/go/storage v1.51.0 h1:ZVZ11zCiD7b3k+cH5lQs/qcNaoSz3U9I0jgwVzqDlCw= +cloud.google.com/go/storage v1.51.0/go.mod h1:YEJfu/Ki3i5oHC/7jyTgsGZwdQ8P9hqMqvpi5kRKGgc= 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= @@ -705,12 +705,12 @@ github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.2 h1:DBjmt6/otSdULyJdVg2 github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.2/go.mod h1:dppbR7CwXD4pgtV9t3wD1812RaLDcBjtblcDF5f1vI0= github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 h1:3c8yed4lgqTt+oTQ+JNMDo+F4xprBf+O/il4ZC0nRLw= github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.49.0 h1:o90wcURuxekmXrtxmYWTyNla0+ZEHhud6DI1ZTxd1vI= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.49.0/go.mod h1:6fTWu4m3jocfUZLYF5KsZC1TUfRvEjs7lM4crme/irw= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.49.0 h1:jJKWl98inONJAr/IZrdFQUWcwUO95DLY1XMD1ZIut+g= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.49.0/go.mod h1:l2fIqmwB+FKSfvn3bAD/0i+AXAxhIZjTK2svT/mgUXs= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.49.0 h1:GYUJLfvd++4DMuMhCFLgLXvFwofIxh/qOwoGuS/LTew= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.49.0/go.mod h1:wRbFgBQUVm1YXrvWKofAEmq9HNJTDphbAaJSSX01KUI= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.51.0 h1:fYE9p3esPxA/C0rQ0AHhP0drtPXDRhaWiwg1DPqO7IU= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.51.0/go.mod h1:BnBReJLvVYx2CS/UHOgVz2BXKXD9wsQPxZug20nZhd0= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.51.0 h1:OqVGm6Ei3x5+yZmSJG1Mh2NwHvpVmZ08CB5qJhT9Nuk= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.51.0/go.mod h1:SZiPHWGOOk3bl8tkevxkoiwPgsIl6CwrWcbwjfHZpdM= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.51.0 h1:6/0iUd0xrnX7qt+mLNRwg5c0PGv8wpE8K90ryANQwMI= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.51.0/go.mod h1:otE2jQekW/PqXk1Awf5lmfokJx4uwuqcj1ab5SpGeW0= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0cBrbBpGY/8hQs= github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= @@ -1058,8 +1058,9 @@ github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= +github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/go-replayers/grpcreplay v1.1.0/go.mod h1:qzAvJ8/wi57zq7gWqaE6AwLM6miiXUQwP1S+I9icmhk= github.com/google/go-replayers/httpreplay v1.1.1/go.mod h1:gN9GeLIs7l6NUoVaSSnv2RiqK1NiwAmD0MrKeC9IIks= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= @@ -1109,8 +1110,8 @@ github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY github.com/googleapis/enterprise-certificate-proxy v0.2.0/go.mod h1:8C0jb7/mgJe/9KK8Lm7X9ctZC2t60YyIpYEI16jx0Qg= github.com/googleapis/enterprise-certificate-proxy v0.2.1/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= -github.com/googleapis/enterprise-certificate-proxy v0.3.4 h1:XYIDZApgAnrN1c855gTgghdIA6Stxb52D5RnLI1SLyw= -github.com/googleapis/enterprise-certificate-proxy v0.3.4/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= +github.com/googleapis/enterprise-certificate-proxy v0.3.5 h1:VgzTY2jogw3xt39CusEnFJWm7rlsq5yL5q9XdLOuP5g= +github.com/googleapis/enterprise-certificate-proxy v0.3.5/go.mod h1:MkHOF77EYAE7qfSuSS9PU6g4Nt4e11cnsDUowfwewLA= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= @@ -1466,12 +1467,12 @@ go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= -go.opentelemetry.io/contrib/detectors/gcp v1.33.0 h1:FVPoXEoILwgbZUu4X7YSgsESsAmGRgoYcnXkzgQPhP4= -go.opentelemetry.io/contrib/detectors/gcp v1.33.0/go.mod h1:ZHrLmr4ikK2AwRj9QL+c9s2SOlgoSRyMpNVzUj2fZqI= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.58.0 h1:PS8wXpbyaDJQ2VDHHncMe9Vct0Zn1fEjpsjrLxGJoSc= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.58.0/go.mod h1:HDBUsEjOuRC0EzKZ1bSaRGZWUBAzo+MhAcUUORSr4D0= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.58.0 h1:yd02MEjBdJkG3uabWP9apV+OuWRIXGDuJEUJbOHmCFU= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.58.0/go.mod h1:umTcuxiv1n/s/S6/c2AT/g2CQ7u5C59sHDNmfSwgz7Q= +go.opentelemetry.io/contrib/detectors/gcp v1.34.0 h1:JRxssobiPg23otYU5SbWtQC//snGVIM3Tx6QRzlQBao= +go.opentelemetry.io/contrib/detectors/gcp v1.34.0/go.mod h1:cV4BMFcscUR/ckqLkbfQmF0PRsq8w/lMGzdbCSveBHo= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0 h1:rgMkmiGfix9vFJDcDi1PK8WEQP4FLQwLDfhp5ZLpFeE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.59.0/go.mod h1:ijPqXp5P6IRRByFVVg9DY8P5HkxkHE5ARIa+86aXPf4= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.59.0 h1:CV7UdSGJt/Ao6Gp4CXckLxVRRsRgDHoI8XjbL3PDl8s= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.59.0/go.mod h1:FRmFuRJfag1IZ2dPkHnEoSFVgTVPUd2qf5Vi69hLb8I= go.opentelemetry.io/otel v1.34.0 h1:zRLXxLCgL1WyKsPVrgbSdMN4c0FMkDAskSTQP+0hdUY= go.opentelemetry.io/otel v1.34.0/go.mod h1:OWFPOQ+h4G8xpyjgqo4SxJYdDQ/qmRH+wivy7zzx9oI= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.33.0 h1:Vh5HayB/0HHfOQA7Ctx69E/Y/DcQSMPpKANYVMQ7fBA= @@ -1484,8 +1485,8 @@ go.opentelemetry.io/otel/metric v1.34.0 h1:+eTR3U0MyfWjRDhmFMxe2SsW64QrZ84AOhvqS go.opentelemetry.io/otel/metric v1.34.0/go.mod h1:CEDrp0fy2D0MvkXE+dPV7cMi8tWZwX3dmaIhwPOaqHE= go.opentelemetry.io/otel/sdk v1.34.0 h1:95zS4k/2GOy069d321O8jWgYsW3MzVV+KuSPKp7Wr1A= go.opentelemetry.io/otel/sdk v1.34.0/go.mod h1:0e/pNiaMAqaykJGKbi+tSjWfNNHMTxoC9qANsCzbyxU= -go.opentelemetry.io/otel/sdk/metric v1.33.0 h1:Gs5VK9/WUJhNXZgn8MR6ITatvAmKeIuCtNbsP3JkNqU= -go.opentelemetry.io/otel/sdk/metric v1.33.0/go.mod h1:dL5ykHZmm1B1nVRk9dDjChwDmt81MjVp3gLkQRwKf/Q= +go.opentelemetry.io/otel/sdk/metric v1.34.0 h1:5CeK9ujjbFVL5c1PhLuStg1wxA7vQv7ce1EK0Gyvahk= +go.opentelemetry.io/otel/sdk/metric v1.34.0/go.mod h1:jQ/r8Ze28zRKoNRdkjCZxfs6YvBTG1+YIqyFVFYec5w= go.opentelemetry.io/otel/trace v1.34.0 h1:+ouXS2V8Rd4hp4580a8q23bg0azF2nI8cqLYnC8mh/k= go.opentelemetry.io/otel/trace v1.34.0/go.mod h1:Svm7lSjQD7kG7KJ/MUHPVXSDGz2OX4h0M2jHBhmSfRE= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= @@ -2028,8 +2029,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.221.0 h1:qzaJfLhDsbMeFee8zBRdt/Nc+xmOuafD/dbdgGfutOU= -google.golang.org/api v0.221.0/go.mod h1:7sOU2+TL4TxUTdbi0gWgAIg7tH5qBXxoyhtL+9x3biQ= +google.golang.org/api v0.224.0 h1:Ir4UPtDsNiwIOHdExr3fAj4xZ42QjK7uQte3lORLJwU= +google.golang.org/api v0.224.0/go.mod h1:3V39my2xAGkodXy0vEqcEtkqgw2GtrFL5WuBZlCTCOQ= 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= @@ -2188,12 +2189,12 @@ google.golang.org/genproto v0.0.0-20230323212658-478b75c54725/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230330154414-c0448cd141ea/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOlWu4KYeJZffbWgBkS1YFobzKbLVfK69pe0Ak= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= -google.golang.org/genproto v0.0.0-20250122153221-138b5a5a4fd4 h1:Pw6WnI9W/LIdRxqK7T6XGugGbHIRl5Q7q3BssH6xk4s= -google.golang.org/genproto v0.0.0-20250122153221-138b5a5a4fd4/go.mod h1:qbZzneIOXSq+KFAFut9krLfRLZiFLzZL5u2t8SV83EE= -google.golang.org/genproto/googleapis/api v0.0.0-20250127172529-29210b9bc287 h1:A2ni10G3UlplFrWdCDJTl7D7mJ7GSRm37S+PDimaKRw= -google.golang.org/genproto/googleapis/api v0.0.0-20250127172529-29210b9bc287/go.mod h1:iYONQfRdizDB8JJBybql13nArx91jcUk7zCXEsOofM4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6 h1:2duwAxN2+k0xLNpjnHTXoMUgnv6VPSp5fiqTuwSxjmI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6/go.mod h1:8BS3B93F/U1juMFq9+EDk+qOT5CO1R9IzXxG3PTqiRk= +google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb h1:ITgPrl429bc6+2ZraNSzMDk3I95nmQln2fuPstKwFDE= +google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb/go.mod h1:sAo5UzpjUwgFBCzupwhcLcxHVDK7vG5IqI30YnwX2eE= +google.golang.org/genproto/googleapis/api v0.0.0-20250303144028-a0af3efb3deb h1:p31xT4yrYrSM/G4Sn2+TNUkVhFCbG9y8itM2S6Th950= +google.golang.org/genproto/googleapis/api v0.0.0-20250303144028-a0af3efb3deb/go.mod h1:jbe3Bkdp+Dh2IrslsFCklNhweNTBgSYanP1UXhJDhKg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250303144028-a0af3efb3deb h1:TLPQVbx1GJ8VKZxz52VAxl1EBgKXXbTiU9Fc5fZeLn4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250303144028-a0af3efb3deb/go.mod h1:LuRYeWDFV6WOn90g357N17oMCaxpgCnbi/44qJvDn2I= 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= @@ -2236,8 +2237,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.70.0 h1:pWFv03aZoHzlRKHWicjsZytKAiYCtNS0dHbXnIdq7jQ= -google.golang.org/grpc v1.70.0/go.mod h1:ofIJqVKDXx/JiXrwr2IG4/zwdH9txy3IlF40RmcJSQw= +google.golang.org/grpc v1.71.0 h1:kF77BGdPTQ4/JZWMlb9VpJ5pa25aqvVqogsxNHHdeBg= +google.golang.org/grpc v1.71.0/go.mod h1:H0GRtasmQOh9LkFoCPDu3ZrwUtD1YGE+b2vYBYd/8Ec= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= From 55a6a67833e61b52c66a76b4c917f9f010adbf8d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 19 Mar 2025 10:44:11 -0700 Subject: [PATCH 35/48] Bump serialize-javascript and mocha in /sdks/typescript (#34012) Bumps [serialize-javascript](https://github.com/yahoo/serialize-javascript) to 6.0.2 and updates ancestor dependency [mocha](https://github.com/mochajs/mocha). These dependencies need to be updated together. Updates `serialize-javascript` from 6.0.0 to 6.0.2 - [Release notes](https://github.com/yahoo/serialize-javascript/releases) - [Commits](https://github.com/yahoo/serialize-javascript/compare/v6.0.0...v6.0.2) Updates `mocha` from 9.2.2 to 11.1.0 - [Release notes](https://github.com/mochajs/mocha/releases) - [Changelog](https://github.com/mochajs/mocha/blob/main/CHANGELOG.md) - [Commits](https://github.com/mochajs/mocha/compare/v9.2.2...v11.1.0) --- updated-dependencies: - dependency-name: serialize-javascript dependency-type: indirect - dependency-name: mocha dependency-type: direct:development ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/typescript/package-lock.json | 1008 +++++++++++++++++++---------- sdks/typescript/package.json | 2 +- 2 files changed, 683 insertions(+), 327 deletions(-) diff --git a/sdks/typescript/package-lock.json b/sdks/typescript/package-lock.json index 4b58d334c977..fb6023480679 100644 --- a/sdks/typescript/package-lock.json +++ b/sdks/typescript/package-lock.json @@ -1,12 +1,12 @@ { "name": "apache-beam", - "version": "2.58.0-SNAPSHOT", + "version": "2.64.0-SNAPSHOT", "lockfileVersion": 2, "requires": true, "packages": { "": { "name": "apache-beam", - "version": "2.58.0-SNAPSHOT", + "version": "2.64.0-SNAPSHOT", "dependencies": { "@google-cloud/pubsub": "^2.19.4", "@grpc/grpc-js": "~1.4.6", @@ -38,7 +38,7 @@ "eslint": "^8.15.0", "istanbul": "^0.4.5", "js-yaml": "^4.1.0", - "mocha": "^9.1.3", + "mocha": "^11.1.0", "prettier": "^3.1.1", "typedoc": "^0.23.23", "typescript": "4.7" @@ -239,6 +239,109 @@ "integrity": "sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==", "dev": true }, + "node_modules/@isaacs/cliui": { + "version": "8.0.2", + "resolved": "https://registry.npmjs.org/@isaacs/cliui/-/cliui-8.0.2.tgz", + "integrity": "sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA==", + "dev": true, + "license": "ISC", + "dependencies": { + "string-width": "^5.1.2", + "string-width-cjs": "npm:string-width@^4.2.0", + "strip-ansi": "^7.0.1", + "strip-ansi-cjs": "npm:strip-ansi@^6.0.1", + "wrap-ansi": "^8.1.0", + "wrap-ansi-cjs": "npm:wrap-ansi@^7.0.0" + }, + "engines": { + "node": ">=12" + } + }, + "node_modules/@isaacs/cliui/node_modules/ansi-regex": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-6.1.0.tgz", + "integrity": "sha512-7HSX4QQb4CspciLpVFwyRe79O3xsIZDDLER21kERQ71oaPodF8jL725AgJMFAYbooIqolJoRLuM81SpeUkpkvA==", + "dev": true, + "license": "MIT", + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/chalk/ansi-regex?sponsor=1" + } + }, + "node_modules/@isaacs/cliui/node_modules/ansi-styles": { + "version": "6.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-6.2.1.tgz", + "integrity": "sha512-bN798gFfQX+viw3R7yrGWRqnrN2oRkEkUjjl4JNn4E8GxxbjtG3FbrEIIY3l8/hrwUwIeCZvi4QuOTP4MErVug==", + "dev": true, + "license": "MIT", + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/chalk/ansi-styles?sponsor=1" + } + }, + "node_modules/@isaacs/cliui/node_modules/emoji-regex": { + "version": "9.2.2", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-9.2.2.tgz", + "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==", + "dev": true, + "license": "MIT" + }, + "node_modules/@isaacs/cliui/node_modules/string-width": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-5.1.2.tgz", + "integrity": "sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA==", + "dev": true, + "license": "MIT", + "dependencies": { + "eastasianwidth": "^0.2.0", + "emoji-regex": "^9.2.2", + "strip-ansi": "^7.0.1" + }, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/sponsors/sindresorhus" + } + }, + "node_modules/@isaacs/cliui/node_modules/strip-ansi": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-7.1.0.tgz", + "integrity": "sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ==", + "dev": true, + "license": "MIT", + "dependencies": { + "ansi-regex": "^6.0.1" + }, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/chalk/strip-ansi?sponsor=1" + } + }, + "node_modules/@isaacs/cliui/node_modules/wrap-ansi": { + "version": "8.1.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-8.1.0.tgz", + "integrity": "sha512-si7QWI6zUMq56bESFvagtmzMdGOtoxfR+Sez11Mobfc7tm+VkUckk9bW2UeffTGVUbOksxmSw0AA2gs8g71NCQ==", + "dev": true, + "license": "MIT", + "dependencies": { + "ansi-styles": "^6.1.0", + "string-width": "^5.0.1", + "strip-ansi": "^7.0.1" + }, + "engines": { + "node": ">=12" + }, + "funding": { + "url": "https://github.com/chalk/wrap-ansi?sponsor=1" + } + }, "node_modules/@nodelib/fs.scandir": { "version": "2.1.5", "resolved": "https://registry.npmjs.org/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz", @@ -290,6 +393,17 @@ "node": ">=8.12.0" } }, + "node_modules/@pkgjs/parseargs": { + "version": "0.11.0", + "resolved": "https://registry.npmjs.org/@pkgjs/parseargs/-/parseargs-0.11.0.tgz", + "integrity": "sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg==", + "dev": true, + "license": "MIT", + "optional": true, + "engines": { + "node": ">=14" + } + }, "node_modules/@protobuf-ts/grpc-transport": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/@protobuf-ts/grpc-transport/-/grpc-transport-2.1.0.tgz", @@ -522,29 +636,6 @@ } } }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "dependencies": { - "ms": "2.1.2" - }, - "engines": { - "node": ">=6.0" - }, - "peerDependenciesMeta": { - "supports-color": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - }, "node_modules/@typescript-eslint/parser": { "version": "5.24.0", "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-5.24.0.tgz", @@ -572,29 +663,6 @@ } } }, - "node_modules/@typescript-eslint/parser/node_modules/debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "dependencies": { - "ms": "2.1.2" - }, - "engines": { - "node": ">=6.0" - }, - "peerDependenciesMeta": { - "supports-color": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/parser/node_modules/ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - }, "node_modules/@typescript-eslint/scope-manager": { "version": "5.24.0", "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-5.24.0.tgz", @@ -638,29 +706,6 @@ } } }, - "node_modules/@typescript-eslint/type-utils/node_modules/debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "dependencies": { - "ms": "2.1.2" - }, - "engines": { - "node": ">=6.0" - }, - "peerDependenciesMeta": { - "supports-color": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/type-utils/node_modules/ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - }, "node_modules/@typescript-eslint/types": { "version": "5.24.0", "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-5.24.0.tgz", @@ -701,29 +746,6 @@ } } }, - "node_modules/@typescript-eslint/typescript-estree/node_modules/debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "dependencies": { - "ms": "2.1.2" - }, - "engines": { - "node": ">=6.0" - }, - "peerDependenciesMeta": { - "supports-color": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/typescript-estree/node_modules/ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - }, "node_modules/@typescript-eslint/utils": { "version": "5.24.0", "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-5.24.0.tgz", @@ -765,12 +787,6 @@ "url": "https://opencollective.com/typescript-eslint" } }, - "node_modules/@ungap/promise-all-settled": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@ungap/promise-all-settled/-/promise-all-settled-1.1.2.tgz", - "integrity": "sha512-sL/cEvJWAnClXw0wHk85/2L0G6Sj8UB0Ctc1TEMbKSsmpRosqhwj9gWgFRZSrBr2f9tiXISwNhCPmlfqUqyb9Q==", - "dev": true - }, "node_modules/abbrev": { "version": "1.0.9", "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.0.9.tgz", @@ -855,10 +871,11 @@ } }, "node_modules/ansi-colors": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", - "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.3.tgz", + "integrity": "sha512-/6w/C21Pm1A7aZitlI5Ni/2J6FFQN8i1Cvz3kHABAAbw93v/NlvKdVOqz7CCWz/3iv/JplRSEEZ83XION15ovw==", "dev": true, + "license": "MIT", "engines": { "node": ">=6" } @@ -1306,11 +1323,12 @@ } }, "node_modules/debug": { - "version": "4.3.3", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.3.tgz", - "integrity": "sha512-/zxw5+vh1Tfv+4Qn7a5nsbcJKPaSvCDhojn6FEl9vupwK2VCSDtEiEtqr8DFtzYFOdz63LBkxec7DYuc2jon6Q==", + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.4.0.tgz", + "integrity": "sha512-6WTZ/IxCY/T6BALoZHaE4ctp9xm+Z5kY/pzYaCHRFeyVhojxlrm+46y68HA6hr0TcwEssoxNiDEUJQjfPZ/RYA==", + "license": "MIT", "dependencies": { - "ms": "2.1.2" + "ms": "^2.1.3" }, "engines": { "node": ">=6.0" @@ -1321,11 +1339,6 @@ } } }, - "node_modules/debug/node_modules/ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" - }, "node_modules/decamelize": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/decamelize/-/decamelize-4.0.0.tgz", @@ -1356,10 +1369,11 @@ "dev": true }, "node_modules/diff": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/diff/-/diff-5.0.0.tgz", - "integrity": "sha512-/VTCrvm5Z0JGty/BWHljh+BAiw3IK+2j87NGMu8Nwc/f48WoDAC395uomO9ZD117ZOBaHmkX1oyLvkVM/aIT3w==", + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/diff/-/diff-5.2.0.tgz", + "integrity": "sha512-uIFDxqpRZGZ6ThOk84hEfqWoHx2devRFvpTZcTHur85vImfaxUbTW9Ryh4CpCuDnToOP1CEtXKIgytHBPVff5A==", "dev": true, + "license": "BSD-3-Clause", "engines": { "node": ">=0.3.1" } @@ -1399,6 +1413,13 @@ "stream-shift": "^1.0.0" } }, + "node_modules/eastasianwidth": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/eastasianwidth/-/eastasianwidth-0.2.0.tgz", + "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==", + "dev": true, + "license": "MIT" + }, "node_modules/ecdsa-sig-formatter": { "version": "1.0.11", "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", @@ -1905,6 +1926,23 @@ "integrity": "sha512-WIWGi2L3DyTUvUrwRKgGi9TwxQMUEqPOPQBVi71R96jZXJdFskXEmf54BoZaS1kknGODoIGASGEzBUYdyMCBJg==", "dev": true }, + "node_modules/foreground-child": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/foreground-child/-/foreground-child-3.3.0.tgz", + "integrity": "sha512-Ld2g8rrAyMYFXBhEqMz8ZAHBi4J4uS1i/CxGMDnjyFWddMXLVcDp051DZfu+t7+ab7Wv6SMqpWmyFIj5UbfFvg==", + "dev": true, + "license": "ISC", + "dependencies": { + "cross-spawn": "^7.0.0", + "signal-exit": "^4.0.1" + }, + "engines": { + "node": ">=14" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/fs.realpath": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", @@ -2226,15 +2264,6 @@ "node": ">=10" } }, - "node_modules/growl": { - "version": "1.10.5", - "resolved": "https://registry.npmjs.org/growl/-/growl-1.10.5.tgz", - "integrity": "sha512-qBr4OuELkhPenW6goKVXiv47US3clb3/IbuWF9KNKEijAy9oeHxU9IgzjvJhHkUzhaj7rOUD7+YGWqUjLp5oSA==", - "dev": true, - "engines": { - "node": ">=4.x" - } - }, "node_modules/gtoken": { "version": "5.3.2", "resolved": "https://registry.npmjs.org/gtoken/-/gtoken-5.3.2.tgz", @@ -2646,6 +2675,22 @@ "which": "bin/which" } }, + "node_modules/jackspeak": { + "version": "3.4.3", + "resolved": "https://registry.npmjs.org/jackspeak/-/jackspeak-3.4.3.tgz", + "integrity": "sha512-OGlZQpz2yfahA/Rd1Y8Cd9SIEsqvXkLVoSw/cgwhnhFMDbsQFeZYoJJ7bIZBS9BcamUW96asq/npPWugM+RQBw==", + "dev": true, + "license": "BlueOak-1.0.0", + "dependencies": { + "@isaacs/cliui": "^8.0.2" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + }, + "optionalDependencies": { + "@pkgjs/parseargs": "^0.11.0" + } + }, "node_modules/js-yaml": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", @@ -2858,6 +2903,16 @@ "integrity": "sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==", "dev": true }, + "node_modules/minipass": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.1.2.tgz", + "integrity": "sha512-qOOzS1cBTWYF4BH8fVePDBOO9iptMnGUEZwNc/cMWnTV2nVLZ7VoNWEPHkYczZA0pdoA7dl6e7FL659nX9S2aw==", + "dev": true, + "license": "ISC", + "engines": { + "node": ">=16 || 14 >=14.17" + } + }, "node_modules/mkdirp": { "version": "0.5.5", "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-0.5.5.tgz", @@ -2871,78 +2926,150 @@ } }, "node_modules/mocha": { - "version": "9.2.2", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-9.2.2.tgz", - "integrity": "sha512-L6XC3EdwT6YrIk0yXpavvLkn8h+EU+Y5UcCHKECyMbdUIxyMuZj4bX4U9e1nvnvUUvQVsV2VHQr5zLdcUkhW/g==", - "dev": true, - "dependencies": { - "@ungap/promise-all-settled": "1.1.2", - "ansi-colors": "4.1.1", - "browser-stdout": "1.3.1", - "chokidar": "3.5.3", - "debug": "4.3.3", - "diff": "5.0.0", - "escape-string-regexp": "4.0.0", - "find-up": "5.0.0", - "glob": "7.2.0", - "growl": "1.10.5", - "he": "1.2.0", - "js-yaml": "4.1.0", - "log-symbols": "4.1.0", - "minimatch": "4.2.1", - "ms": "2.1.3", - "nanoid": "3.3.1", - "serialize-javascript": "6.0.0", - "strip-json-comments": "3.1.1", - "supports-color": "8.1.1", - "which": "2.0.2", - "workerpool": "6.2.0", - "yargs": "16.2.0", - "yargs-parser": "20.2.4", - "yargs-unparser": "2.0.0" + "version": "11.1.0", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-11.1.0.tgz", + "integrity": "sha512-8uJR5RTC2NgpY3GrYcgpZrsEd9zKbPDpob1RezyR2upGHRQtHWofmzTMzTMSV6dru3tj5Ukt0+Vnq1qhFEEwAg==", + "dev": true, + "license": "MIT", + "dependencies": { + "ansi-colors": "^4.1.3", + "browser-stdout": "^1.3.1", + "chokidar": "^3.5.3", + "debug": "^4.3.5", + "diff": "^5.2.0", + "escape-string-regexp": "^4.0.0", + "find-up": "^5.0.0", + "glob": "^10.4.5", + "he": "^1.2.0", + "js-yaml": "^4.1.0", + "log-symbols": "^4.1.0", + "minimatch": "^5.1.6", + "ms": "^2.1.3", + "serialize-javascript": "^6.0.2", + "strip-json-comments": "^3.1.1", + "supports-color": "^8.1.1", + "workerpool": "^6.5.1", + "yargs": "^17.7.2", + "yargs-parser": "^21.1.1", + "yargs-unparser": "^2.0.0" }, "bin": { "_mocha": "bin/_mocha", - "mocha": "bin/mocha" + "mocha": "bin/mocha.js" }, "engines": { - "node": ">= 12.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + } + }, + "node_modules/mocha/node_modules/brace-expansion": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-2.0.1.tgz", + "integrity": "sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA==", + "dev": true, + "license": "MIT", + "dependencies": { + "balanced-match": "^1.0.0" + } + }, + "node_modules/mocha/node_modules/cliui": { + "version": "8.0.1", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-8.0.1.tgz", + "integrity": "sha512-BSeNnyus75C4//NQ9gQt1/csTXyo/8Sb+afLAkzAptFuMsod9HFokGNudZpi/oQV73hnVK+sR+5PVRMd+Dr7YQ==", + "dev": true, + "license": "ISC", + "dependencies": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.1", + "wrap-ansi": "^7.0.0" + }, + "engines": { + "node": ">=12" + } + }, + "node_modules/mocha/node_modules/glob": { + "version": "10.4.5", + "resolved": "https://registry.npmjs.org/glob/-/glob-10.4.5.tgz", + "integrity": "sha512-7Bv8RF0k6xjo7d4A/PxYLbUCfb6c+Vpd2/mB2yRDlew7Jb5hEXiCD9ibfO7wpk8i4sevK6DFny9h7EYbM3/sHg==", + "dev": true, + "license": "ISC", + "dependencies": { + "foreground-child": "^3.1.0", + "jackspeak": "^3.1.2", + "minimatch": "^9.0.4", + "minipass": "^7.1.2", + "package-json-from-dist": "^1.0.0", + "path-scurry": "^1.11.1" + }, + "bin": { + "glob": "dist/esm/bin.mjs" }, "funding": { - "type": "opencollective", - "url": "https://opencollective.com/mochajs" + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/mocha/node_modules/glob/node_modules/minimatch": { + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", + "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", + "dev": true, + "license": "ISC", + "dependencies": { + "brace-expansion": "^2.0.1" + }, + "engines": { + "node": ">=16 || 14 >=14.17" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" } }, "node_modules/mocha/node_modules/minimatch": { - "version": "4.2.1", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-4.2.1.tgz", - "integrity": "sha512-9Uq1ChtSZO+Mxa/CL1eGizn2vRn3MlLgzhT0Iz8zaY8NdvxvB0d5QdPFmCKf7JKA9Lerx5vRrnwO03jsSfGG9g==", + "version": "5.1.6", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.1.6.tgz", + "integrity": "sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g==", "dev": true, + "license": "ISC", "dependencies": { - "brace-expansion": "^1.1.7" + "brace-expansion": "^2.0.1" }, "engines": { "node": ">=10" } }, - "node_modules/ms": { - "version": "2.1.3", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", - "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", - "dev": true - }, - "node_modules/nanoid": { - "version": "3.3.1", - "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.1.tgz", - "integrity": "sha512-n6Vs/3KGyxPQd6uO0eH4Bv0ojGSUvuLlIHtC3Y0kEO23YRge8H9x1GCzLn28YX0H66pMkxuaeESFq4tKISKwdw==", + "node_modules/mocha/node_modules/yargs": { + "version": "17.7.2", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-17.7.2.tgz", + "integrity": "sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w==", "dev": true, - "bin": { - "nanoid": "bin/nanoid.cjs" + "license": "MIT", + "dependencies": { + "cliui": "^8.0.1", + "escalade": "^3.1.1", + "get-caller-file": "^2.0.5", + "require-directory": "^2.1.1", + "string-width": "^4.2.3", + "y18n": "^5.0.5", + "yargs-parser": "^21.1.1" }, "engines": { - "node": "^10 || ^12 || ^13.7 || ^14 || >=15.0.1" + "node": ">=12" + } + }, + "node_modules/mocha/node_modules/yargs-parser": { + "version": "21.1.1", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-21.1.1.tgz", + "integrity": "sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw==", + "dev": true, + "license": "ISC", + "engines": { + "node": ">=12" } }, + "node_modules/ms": { + "version": "2.1.3", + "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==" + }, "node_modules/natural-compare": { "version": "1.4.0", "resolved": "https://registry.npmjs.org/natural-compare/-/natural-compare-1.4.0.tgz", @@ -3110,6 +3237,13 @@ "node": ">=8" } }, + "node_modules/package-json-from-dist": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/package-json-from-dist/-/package-json-from-dist-1.0.1.tgz", + "integrity": "sha512-UEZIS3/by4OC8vL3P2dTXRETpebLI2NiI5vIrjaD/5UtrkFX/tNbwjTSRAGC/+7CAo2pIcBaRgWmcBBHcsaCIw==", + "dev": true, + "license": "BlueOak-1.0.0" + }, "node_modules/parent-module": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/parent-module/-/parent-module-1.0.1.tgz", @@ -3154,6 +3288,30 @@ "resolved": "https://registry.npmjs.org/path-parse/-/path-parse-1.0.7.tgz", "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==" }, + "node_modules/path-scurry": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/path-scurry/-/path-scurry-1.11.1.tgz", + "integrity": "sha512-Xa4Nw17FS9ApQFJ9umLiJS4orGjm7ZzwUrwamcGQuHSzDyth9boKDaycYdDcZDuqYATXw4HFXgaqWTctW/v1HA==", + "dev": true, + "license": "BlueOak-1.0.0", + "dependencies": { + "lru-cache": "^10.2.0", + "minipass": "^5.0.0 || ^6.0.2 || ^7.0.0" + }, + "engines": { + "node": ">=16 || 14 >=14.18" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, + "node_modules/path-scurry/node_modules/lru-cache": { + "version": "10.4.3", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-10.4.3.tgz", + "integrity": "sha512-JNAzZcXrCt42VGLuYz0zfAzDfAvJWW6AfYlDBQyDV5DClI2m5sAmK+OIO7s59XfsRsWHp02jAJrRadPRGTt6SQ==", + "dev": true, + "license": "ISC" + }, "node_modules/path-type": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", @@ -3279,6 +3437,7 @@ "resolved": "https://registry.npmjs.org/randombytes/-/randombytes-2.1.0.tgz", "integrity": "sha512-vYl3iOX+4CKUWuxGi9Ukhie6fsqXqS9FE2Zaic4tNFD2N2QQaXOMFbuKK4QmDHC0JO6B1Zp41J0LpT0oR68amQ==", "dev": true, + "license": "MIT", "dependencies": { "safe-buffer": "^5.1.0" } @@ -3457,10 +3616,11 @@ "integrity": "sha512-FJkTECOkJ1jiRdy/BF24lR9AU+siChyYEYrLNy1ZPSz5bas4ZxslM/Fm0VwActiL0zTo1UA5XE2NRubM78+i4g==" }, "node_modules/serialize-javascript": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", - "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, + "license": "BSD-3-Clause", "dependencies": { "randombytes": "^2.1.0" } @@ -3497,6 +3657,19 @@ "vscode-textmate": "^6.0.0" } }, + "node_modules/signal-exit": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-4.1.0.tgz", + "integrity": "sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw==", + "dev": true, + "license": "ISC", + "engines": { + "node": ">=14" + }, + "funding": { + "url": "https://github.com/sponsors/isaacs" + } + }, "node_modules/slash": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/slash/-/slash-3.0.0.tgz", @@ -3560,6 +3733,22 @@ "node": ">=8" } }, + "node_modules/string-width-cjs": { + "name": "string-width", + "version": "4.2.3", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", + "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "dev": true, + "license": "MIT", + "dependencies": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/strip-ansi": { "version": "6.0.1", "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", @@ -3571,6 +3760,20 @@ "node": ">=8" } }, + "node_modules/strip-ansi-cjs": { + "name": "strip-ansi", + "version": "6.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", + "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "dev": true, + "license": "MIT", + "dependencies": { + "ansi-regex": "^5.0.1" + }, + "engines": { + "node": ">=8" + } + }, "node_modules/strip-json-comments": { "version": "3.1.1", "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", @@ -3972,10 +4175,11 @@ "dev": true }, "node_modules/workerpool": { - "version": "6.2.0", - "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.2.0.tgz", - "integrity": "sha512-Rsk5qQHJ9eowMH28Jwhe8HEbmdYDX4lwoMWshiCXugjtHqMD9ZbiqSDLxcsfdqsETPzVUtX5s1Z5kStiIM6l4A==", - "dev": true + "version": "6.5.1", + "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.5.1.tgz", + "integrity": "sha512-Fs4dNYcsdpYSAfVxhnl1L5zTksjvOJxtC5hzMNl+1t9B8hTJTdKDyZ5ju7ztgPy+ft9tBFXoOlDNiOT9WUXZlA==", + "dev": true, + "license": "Apache-2.0" }, "node_modules/wrap-ansi": { "version": "7.0.0", @@ -3993,6 +4197,25 @@ "url": "https://github.com/chalk/wrap-ansi?sponsor=1" } }, + "node_modules/wrap-ansi-cjs": { + "name": "wrap-ansi", + "version": "7.0.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", + "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "dev": true, + "license": "MIT", + "dependencies": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + }, + "engines": { + "node": ">=10" + }, + "funding": { + "url": "https://github.com/chalk/wrap-ansi?sponsor=1" + } + }, "node_modules/wrappy": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", @@ -4227,6 +4450,71 @@ "integrity": "sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==", "dev": true }, + "@isaacs/cliui": { + "version": "8.0.2", + "resolved": "https://registry.npmjs.org/@isaacs/cliui/-/cliui-8.0.2.tgz", + "integrity": "sha512-O8jcjabXaleOG9DQ0+ARXWZBTfnP4WNAqzuiJK7ll44AmxGKv/J2M4TPjxjY3znBCfvBXFzucm1twdyFybFqEA==", + "dev": true, + "requires": { + "string-width": "^5.1.2", + "string-width-cjs": "npm:string-width@^4.2.0", + "strip-ansi": "^7.0.1", + "strip-ansi-cjs": "npm:strip-ansi@^6.0.1", + "wrap-ansi": "^8.1.0", + "wrap-ansi-cjs": "npm:wrap-ansi@^7.0.0" + }, + "dependencies": { + "ansi-regex": { + "version": "6.1.0", + "resolved": "https://registry.npmjs.org/ansi-regex/-/ansi-regex-6.1.0.tgz", + "integrity": "sha512-7HSX4QQb4CspciLpVFwyRe79O3xsIZDDLER21kERQ71oaPodF8jL725AgJMFAYbooIqolJoRLuM81SpeUkpkvA==", + "dev": true + }, + "ansi-styles": { + "version": "6.2.1", + "resolved": "https://registry.npmjs.org/ansi-styles/-/ansi-styles-6.2.1.tgz", + "integrity": "sha512-bN798gFfQX+viw3R7yrGWRqnrN2oRkEkUjjl4JNn4E8GxxbjtG3FbrEIIY3l8/hrwUwIeCZvi4QuOTP4MErVug==", + "dev": true + }, + "emoji-regex": { + "version": "9.2.2", + "resolved": "https://registry.npmjs.org/emoji-regex/-/emoji-regex-9.2.2.tgz", + "integrity": "sha512-L18DaJsXSUk2+42pv8mLs5jJT2hqFkFE4j21wOmgbUqsZ2hL72NsUU785g9RXgo3s0ZNgVl42TiHp3ZtOv/Vyg==", + "dev": true + }, + "string-width": { + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-5.1.2.tgz", + "integrity": "sha512-HnLOCR3vjcY8beoNLtcjZ5/nxn2afmME6lhrDrebokqMap+XbeW8n9TXpPDOqdGK5qcI3oT0GKTW6wC7EMiVqA==", + "dev": true, + "requires": { + "eastasianwidth": "^0.2.0", + "emoji-regex": "^9.2.2", + "strip-ansi": "^7.0.1" + } + }, + "strip-ansi": { + "version": "7.1.0", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-7.1.0.tgz", + "integrity": "sha512-iq6eVVI64nQQTRYq2KtEg2d2uU7LElhTJwsH4YzIHZshxlgZms/wIc4VoDQTlG/IvVIrBKG06CrZnp0qv7hkcQ==", + "dev": true, + "requires": { + "ansi-regex": "^6.0.1" + } + }, + "wrap-ansi": { + "version": "8.1.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-8.1.0.tgz", + "integrity": "sha512-si7QWI6zUMq56bESFvagtmzMdGOtoxfR+Sez11Mobfc7tm+VkUckk9bW2UeffTGVUbOksxmSw0AA2gs8g71NCQ==", + "dev": true, + "requires": { + "ansi-styles": "^6.1.0", + "string-width": "^5.0.1", + "strip-ansi": "^7.0.1" + } + } + } + }, "@nodelib/fs.scandir": { "version": "2.1.5", "resolved": "https://registry.npmjs.org/@nodelib/fs.scandir/-/fs.scandir-2.1.5.tgz", @@ -4263,6 +4551,13 @@ "resolved": "https://registry.npmjs.org/@opentelemetry/semantic-conventions/-/semantic-conventions-1.2.0.tgz", "integrity": "sha512-BNKB9fiYVghALJzCuWO3eNYfdTExPVK4ykrtmfNfy0A6UWYhOYjGMXifUmkunDJNL8ju9tBobo8jF0WR9zGy1Q==" }, + "@pkgjs/parseargs": { + "version": "0.11.0", + "resolved": "https://registry.npmjs.org/@pkgjs/parseargs/-/parseargs-0.11.0.tgz", + "integrity": "sha512-+1VkjdD0QBLPodGrJUeqarH8VAIvQODIbwh9XpP5Syisf7YoQgsJKPNFoqqLQlu+VQ/tVSshMR6loPMn8U+dPg==", + "dev": true, + "optional": true + }, "@protobuf-ts/grpc-transport": { "version": "2.1.0", "resolved": "https://registry.npmjs.org/@protobuf-ts/grpc-transport/-/grpc-transport-2.1.0.tgz", @@ -4454,23 +4749,6 @@ "regexpp": "^3.2.0", "semver": "^7.3.7", "tsutils": "^3.21.0" - }, - "dependencies": { - "debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "requires": { - "ms": "2.1.2" - } - }, - "ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - } } }, "@typescript-eslint/parser": { @@ -4483,23 +4761,6 @@ "@typescript-eslint/types": "5.24.0", "@typescript-eslint/typescript-estree": "5.24.0", "debug": "^4.3.4" - }, - "dependencies": { - "debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "requires": { - "ms": "2.1.2" - } - }, - "ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - } } }, "@typescript-eslint/scope-manager": { @@ -4521,23 +4782,6 @@ "@typescript-eslint/utils": "5.24.0", "debug": "^4.3.4", "tsutils": "^3.21.0" - }, - "dependencies": { - "debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "requires": { - "ms": "2.1.2" - } - }, - "ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - } } }, "@typescript-eslint/types": { @@ -4559,23 +4803,6 @@ "is-glob": "^4.0.3", "semver": "^7.3.7", "tsutils": "^3.21.0" - }, - "dependencies": { - "debug": { - "version": "4.3.4", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.4.tgz", - "integrity": "sha512-PRWFHuSU3eDtQJPvnNY7Jcket1j0t5OuOsFzPPzsekD52Zl8qUfFIPEiswXqIvHWGVHOgX+7G/vCNNhehwxfkQ==", - "dev": true, - "requires": { - "ms": "2.1.2" - } - }, - "ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==", - "dev": true - } } }, "@typescript-eslint/utils": { @@ -4602,12 +4829,6 @@ "eslint-visitor-keys": "^3.3.0" } }, - "@ungap/promise-all-settled": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@ungap/promise-all-settled/-/promise-all-settled-1.1.2.tgz", - "integrity": "sha512-sL/cEvJWAnClXw0wHk85/2L0G6Sj8UB0Ctc1TEMbKSsmpRosqhwj9gWgFRZSrBr2f9tiXISwNhCPmlfqUqyb9Q==", - "dev": true - }, "abbrev": { "version": "1.0.9", "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.0.9.tgz", @@ -4668,9 +4889,9 @@ "optional": true }, "ansi-colors": { - "version": "4.1.1", - "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.1.tgz", - "integrity": "sha512-JoX0apGbHaUJBNl6yF+p6JAFYZ666/hhCGKN5t9QFjbJQKUU/g8MNbFDbvfrgKXvI1QpZplPOnwIo99lX/AAmA==", + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/ansi-colors/-/ansi-colors-4.1.3.tgz", + "integrity": "sha512-/6w/C21Pm1A7aZitlI5Ni/2J6FFQN8i1Cvz3kHABAAbw93v/NlvKdVOqz7CCWz/3iv/JplRSEEZ83XION15ovw==", "dev": true }, "ansi-regex": { @@ -4988,18 +5209,11 @@ "integrity": "sha512-8d35hViGYx/QH0icHYCeLmsLmMUheMmTyV9Fcm6gvNwdw31yXXH+O85sOBJ+OLnLQMKZowvpKb6FgMIQjcpvQw==" }, "debug": { - "version": "4.3.3", - "resolved": "https://registry.npmjs.org/debug/-/debug-4.3.3.tgz", - "integrity": "sha512-/zxw5+vh1Tfv+4Qn7a5nsbcJKPaSvCDhojn6FEl9vupwK2VCSDtEiEtqr8DFtzYFOdz63LBkxec7DYuc2jon6Q==", + "version": "4.4.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-4.4.0.tgz", + "integrity": "sha512-6WTZ/IxCY/T6BALoZHaE4ctp9xm+Z5kY/pzYaCHRFeyVhojxlrm+46y68HA6hr0TcwEssoxNiDEUJQjfPZ/RYA==", "requires": { - "ms": "2.1.2" - }, - "dependencies": { - "ms": { - "version": "2.1.2", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.2.tgz", - "integrity": "sha512-sGkPx+VjMtmA6MX27oA4FBFELFCZZ4S4XqeGOXCv68tT+jb3vk/RyaKWP0PTKyWtmLSM0b+adUTEvbs1PEaH2w==" - } + "ms": "^2.1.3" } }, "decamelize": { @@ -5023,9 +5237,9 @@ "dev": true }, "diff": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/diff/-/diff-5.0.0.tgz", - "integrity": "sha512-/VTCrvm5Z0JGty/BWHljh+BAiw3IK+2j87NGMu8Nwc/f48WoDAC395uomO9ZD117ZOBaHmkX1oyLvkVM/aIT3w==", + "version": "5.2.0", + "resolved": "https://registry.npmjs.org/diff/-/diff-5.2.0.tgz", + "integrity": "sha512-uIFDxqpRZGZ6ThOk84hEfqWoHx2devRFvpTZcTHur85vImfaxUbTW9Ryh4CpCuDnToOP1CEtXKIgytHBPVff5A==", "dev": true }, "dir-glob": { @@ -5057,6 +5271,12 @@ "stream-shift": "^1.0.0" } }, + "eastasianwidth": { + "version": "0.2.0", + "resolved": "https://registry.npmjs.org/eastasianwidth/-/eastasianwidth-0.2.0.tgz", + "integrity": "sha512-I88TYZWc9XiYHRQ4/3c5rjjfgkjhLyW2luGIheGERbNQ6OY7yTybanSpDXZa8y7VUP9YmDcYa+eyq4ca7iLqWA==", + "dev": true + }, "ecdsa-sig-formatter": { "version": "1.0.11", "resolved": "https://registry.npmjs.org/ecdsa-sig-formatter/-/ecdsa-sig-formatter-1.0.11.tgz", @@ -5447,6 +5667,16 @@ "integrity": "sha512-WIWGi2L3DyTUvUrwRKgGi9TwxQMUEqPOPQBVi71R96jZXJdFskXEmf54BoZaS1kknGODoIGASGEzBUYdyMCBJg==", "dev": true }, + "foreground-child": { + "version": "3.3.0", + "resolved": "https://registry.npmjs.org/foreground-child/-/foreground-child-3.3.0.tgz", + "integrity": "sha512-Ld2g8rrAyMYFXBhEqMz8ZAHBi4J4uS1i/CxGMDnjyFWddMXLVcDp051DZfu+t7+ab7Wv6SMqpWmyFIj5UbfFvg==", + "dev": true, + "requires": { + "cross-spawn": "^7.0.0", + "signal-exit": "^4.0.1" + } + }, "fs.realpath": { "version": "1.0.0", "resolved": "https://registry.npmjs.org/fs.realpath/-/fs.realpath-1.0.0.tgz", @@ -5692,12 +5922,6 @@ "node-forge": "^1.3.1" } }, - "growl": { - "version": "1.10.5", - "resolved": "https://registry.npmjs.org/growl/-/growl-1.10.5.tgz", - "integrity": "sha512-qBr4OuELkhPenW6goKVXiv47US3clb3/IbuWF9KNKEijAy9oeHxU9IgzjvJhHkUzhaj7rOUD7+YGWqUjLp5oSA==", - "dev": true - }, "gtoken": { "version": "5.3.2", "resolved": "https://registry.npmjs.org/gtoken/-/gtoken-5.3.2.tgz", @@ -5991,6 +6215,16 @@ } } }, + "jackspeak": { + "version": "3.4.3", + "resolved": "https://registry.npmjs.org/jackspeak/-/jackspeak-3.4.3.tgz", + "integrity": "sha512-OGlZQpz2yfahA/Rd1Y8Cd9SIEsqvXkLVoSw/cgwhnhFMDbsQFeZYoJJ7bIZBS9BcamUW96asq/npPWugM+RQBw==", + "dev": true, + "requires": { + "@isaacs/cliui": "^8.0.2", + "@pkgjs/parseargs": "^0.11.0" + } + }, "js-yaml": { "version": "4.1.0", "resolved": "https://registry.npmjs.org/js-yaml/-/js-yaml-4.1.0.tgz", @@ -6167,6 +6401,12 @@ "integrity": "sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==", "dev": true }, + "minipass": { + "version": "7.1.2", + "resolved": "https://registry.npmjs.org/minipass/-/minipass-7.1.2.tgz", + "integrity": "sha512-qOOzS1cBTWYF4BH8fVePDBOO9iptMnGUEZwNc/cMWnTV2nVLZ7VoNWEPHkYczZA0pdoA7dl6e7FL659nX9S2aw==", + "dev": true + }, "mkdirp": { "version": "0.5.5", "resolved": "https://registry.npmjs.org/mkdirp/-/mkdirp-0.5.5.tgz", @@ -6177,59 +6417,114 @@ } }, "mocha": { - "version": "9.2.2", - "resolved": "https://registry.npmjs.org/mocha/-/mocha-9.2.2.tgz", - "integrity": "sha512-L6XC3EdwT6YrIk0yXpavvLkn8h+EU+Y5UcCHKECyMbdUIxyMuZj4bX4U9e1nvnvUUvQVsV2VHQr5zLdcUkhW/g==", - "dev": true, - "requires": { - "@ungap/promise-all-settled": "1.1.2", - "ansi-colors": "4.1.1", - "browser-stdout": "1.3.1", - "chokidar": "3.5.3", - "debug": "4.3.3", - "diff": "5.0.0", - "escape-string-regexp": "4.0.0", - "find-up": "5.0.0", - "glob": "7.2.0", - "growl": "1.10.5", - "he": "1.2.0", - "js-yaml": "4.1.0", - "log-symbols": "4.1.0", - "minimatch": "4.2.1", - "ms": "2.1.3", - "nanoid": "3.3.1", - "serialize-javascript": "6.0.0", - "strip-json-comments": "3.1.1", - "supports-color": "8.1.1", - "which": "2.0.2", - "workerpool": "6.2.0", - "yargs": "16.2.0", - "yargs-parser": "20.2.4", - "yargs-unparser": "2.0.0" + "version": "11.1.0", + "resolved": "https://registry.npmjs.org/mocha/-/mocha-11.1.0.tgz", + "integrity": "sha512-8uJR5RTC2NgpY3GrYcgpZrsEd9zKbPDpob1RezyR2upGHRQtHWofmzTMzTMSV6dru3tj5Ukt0+Vnq1qhFEEwAg==", + "dev": true, + "requires": { + "ansi-colors": "^4.1.3", + "browser-stdout": "^1.3.1", + "chokidar": "^3.5.3", + "debug": "^4.3.5", + "diff": "^5.2.0", + "escape-string-regexp": "^4.0.0", + "find-up": "^5.0.0", + "glob": "^10.4.5", + "he": "^1.2.0", + "js-yaml": "^4.1.0", + "log-symbols": "^4.1.0", + "minimatch": "^5.1.6", + "ms": "^2.1.3", + "serialize-javascript": "^6.0.2", + "strip-json-comments": "^3.1.1", + "supports-color": "^8.1.1", + "workerpool": "^6.5.1", + "yargs": "^17.7.2", + "yargs-parser": "^21.1.1", + "yargs-unparser": "^2.0.0" }, "dependencies": { + "brace-expansion": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/brace-expansion/-/brace-expansion-2.0.1.tgz", + "integrity": "sha512-XnAIvQ8eM+kC6aULx6wuQiwVsnzsi9d3WxzV3FpWTGA19F621kwdbsAcFKXgKUHZWsy+mY6iL1sHTxWEFCytDA==", + "dev": true, + "requires": { + "balanced-match": "^1.0.0" + } + }, + "cliui": { + "version": "8.0.1", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-8.0.1.tgz", + "integrity": "sha512-BSeNnyus75C4//NQ9gQt1/csTXyo/8Sb+afLAkzAptFuMsod9HFokGNudZpi/oQV73hnVK+sR+5PVRMd+Dr7YQ==", + "dev": true, + "requires": { + "string-width": "^4.2.0", + "strip-ansi": "^6.0.1", + "wrap-ansi": "^7.0.0" + } + }, + "glob": { + "version": "10.4.5", + "resolved": "https://registry.npmjs.org/glob/-/glob-10.4.5.tgz", + "integrity": "sha512-7Bv8RF0k6xjo7d4A/PxYLbUCfb6c+Vpd2/mB2yRDlew7Jb5hEXiCD9ibfO7wpk8i4sevK6DFny9h7EYbM3/sHg==", + "dev": true, + "requires": { + "foreground-child": "^3.1.0", + "jackspeak": "^3.1.2", + "minimatch": "^9.0.4", + "minipass": "^7.1.2", + "package-json-from-dist": "^1.0.0", + "path-scurry": "^1.11.1" + }, + "dependencies": { + "minimatch": { + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", + "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", + "dev": true, + "requires": { + "brace-expansion": "^2.0.1" + } + } + } + }, "minimatch": { - "version": "4.2.1", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-4.2.1.tgz", - "integrity": "sha512-9Uq1ChtSZO+Mxa/CL1eGizn2vRn3MlLgzhT0Iz8zaY8NdvxvB0d5QdPFmCKf7JKA9Lerx5vRrnwO03jsSfGG9g==", + "version": "5.1.6", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-5.1.6.tgz", + "integrity": "sha512-lKwV/1brpG6mBUFHtb7NUmtABCb2WZZmm2wNiOA5hAb8VdCS4B3dtMWyvcoViccwAW/COERjXLt0zP1zXUN26g==", "dev": true, "requires": { - "brace-expansion": "^1.1.7" + "brace-expansion": "^2.0.1" } + }, + "yargs": { + "version": "17.7.2", + "resolved": "https://registry.npmjs.org/yargs/-/yargs-17.7.2.tgz", + "integrity": "sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w==", + "dev": true, + "requires": { + "cliui": "^8.0.1", + "escalade": "^3.1.1", + "get-caller-file": "^2.0.5", + "require-directory": "^2.1.1", + "string-width": "^4.2.3", + "y18n": "^5.0.5", + "yargs-parser": "^21.1.1" + } + }, + "yargs-parser": { + "version": "21.1.1", + "resolved": "https://registry.npmjs.org/yargs-parser/-/yargs-parser-21.1.1.tgz", + "integrity": "sha512-tVpsJW7DdjecAiFpbIB1e3qxIQsE6NoPc5/eTdrbbIC4h0LVsWhnoa3g+m2HclBIujHzsxZ4VJVA+GUuc2/LBw==", + "dev": true } } }, "ms": { "version": "2.1.3", "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", - "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", - "dev": true - }, - "nanoid": { - "version": "3.3.1", - "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.1.tgz", - "integrity": "sha512-n6Vs/3KGyxPQd6uO0eH4Bv0ojGSUvuLlIHtC3Y0kEO23YRge8H9x1GCzLn28YX0H66pMkxuaeESFq4tKISKwdw==", - "dev": true + "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==" }, "natural-compare": { "version": "1.4.0", @@ -6345,6 +6640,12 @@ "p-finally": "^1.0.0" } }, + "package-json-from-dist": { + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/package-json-from-dist/-/package-json-from-dist-1.0.1.tgz", + "integrity": "sha512-UEZIS3/by4OC8vL3P2dTXRETpebLI2NiI5vIrjaD/5UtrkFX/tNbwjTSRAGC/+7CAo2pIcBaRgWmcBBHcsaCIw==", + "dev": true + }, "parent-module": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/parent-module/-/parent-module-1.0.1.tgz", @@ -6377,6 +6678,24 @@ "resolved": "https://registry.npmjs.org/path-parse/-/path-parse-1.0.7.tgz", "integrity": "sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw==" }, + "path-scurry": { + "version": "1.11.1", + "resolved": "https://registry.npmjs.org/path-scurry/-/path-scurry-1.11.1.tgz", + "integrity": "sha512-Xa4Nw17FS9ApQFJ9umLiJS4orGjm7ZzwUrwamcGQuHSzDyth9boKDaycYdDcZDuqYATXw4HFXgaqWTctW/v1HA==", + "dev": true, + "requires": { + "lru-cache": "^10.2.0", + "minipass": "^5.0.0 || ^6.0.2 || ^7.0.0" + }, + "dependencies": { + "lru-cache": { + "version": "10.4.3", + "resolved": "https://registry.npmjs.org/lru-cache/-/lru-cache-10.4.3.tgz", + "integrity": "sha512-JNAzZcXrCt42VGLuYz0zfAzDfAvJWW6AfYlDBQyDV5DClI2m5sAmK+OIO7s59XfsRsWHp02jAJrRadPRGTt6SQ==", + "dev": true + } + } + }, "path-type": { "version": "4.0.0", "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", @@ -6566,9 +6885,9 @@ "integrity": "sha512-FJkTECOkJ1jiRdy/BF24lR9AU+siChyYEYrLNy1ZPSz5bas4ZxslM/Fm0VwActiL0zTo1UA5XE2NRubM78+i4g==" }, "serialize-javascript": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.0.tgz", - "integrity": "sha512-Qr3TosvguFt8ePWqsvRfrKyQXIiW+nGbYpy8XK24NQHE83caxWt+mIymTT19DGFbNWNLfEwsrkSmN64lVWB9ag==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, "requires": { "randombytes": "^2.1.0" @@ -6600,6 +6919,12 @@ "vscode-textmate": "^6.0.0" } }, + "signal-exit": { + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/signal-exit/-/signal-exit-4.1.0.tgz", + "integrity": "sha512-bzyZ1e88w9O1iNJbKnOlvYTrWPDl46O1bG0D3XInv+9tkPrxrN8jUUTiFlDkkmKWgn1M6CfIA13SuGqOa9Korw==", + "dev": true + }, "slash": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/slash/-/slash-3.0.0.tgz", @@ -6654,6 +6979,17 @@ "strip-ansi": "^6.0.1" } }, + "string-width-cjs": { + "version": "npm:string-width@4.2.3", + "resolved": "https://registry.npmjs.org/string-width/-/string-width-4.2.3.tgz", + "integrity": "sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==", + "dev": true, + "requires": { + "emoji-regex": "^8.0.0", + "is-fullwidth-code-point": "^3.0.0", + "strip-ansi": "^6.0.1" + } + }, "strip-ansi": { "version": "6.0.1", "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", @@ -6662,6 +6998,15 @@ "ansi-regex": "^5.0.1" } }, + "strip-ansi-cjs": { + "version": "npm:strip-ansi@6.0.1", + "resolved": "https://registry.npmjs.org/strip-ansi/-/strip-ansi-6.0.1.tgz", + "integrity": "sha512-Y38VPSHcqkFrCpFnQ9vuSXmquuv5oXOKpGeT6aGrr3o3Gc9AlVa6JBfUSOCnbxGGZF+/0ooI7KrPuUSztUdU5A==", + "dev": true, + "requires": { + "ansi-regex": "^5.0.1" + } + }, "strip-json-comments": { "version": "3.1.1", "resolved": "https://registry.npmjs.org/strip-json-comments/-/strip-json-comments-3.1.1.tgz", @@ -6950,9 +7295,9 @@ "dev": true }, "workerpool": { - "version": "6.2.0", - "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.2.0.tgz", - "integrity": "sha512-Rsk5qQHJ9eowMH28Jwhe8HEbmdYDX4lwoMWshiCXugjtHqMD9ZbiqSDLxcsfdqsETPzVUtX5s1Z5kStiIM6l4A==", + "version": "6.5.1", + "resolved": "https://registry.npmjs.org/workerpool/-/workerpool-6.5.1.tgz", + "integrity": "sha512-Fs4dNYcsdpYSAfVxhnl1L5zTksjvOJxtC5hzMNl+1t9B8hTJTdKDyZ5ju7ztgPy+ft9tBFXoOlDNiOT9WUXZlA==", "dev": true }, "wrap-ansi": { @@ -6965,6 +7310,17 @@ "strip-ansi": "^6.0.0" } }, + "wrap-ansi-cjs": { + "version": "npm:wrap-ansi@7.0.0", + "resolved": "https://registry.npmjs.org/wrap-ansi/-/wrap-ansi-7.0.0.tgz", + "integrity": "sha512-YVGIj2kamLSTxw6NsZjoBxfSwsn0ycdesmc4p+Q21c5zPuZ1pl+NfxVdxPtdHvmNVOQ6XSYG4AUtyt/Fi7D16Q==", + "dev": true, + "requires": { + "ansi-styles": "^4.0.0", + "string-width": "^4.1.0", + "strip-ansi": "^6.0.0" + } + }, "wrappy": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", diff --git a/sdks/typescript/package.json b/sdks/typescript/package.json index 931146dfb244..731849a50cbd 100644 --- a/sdks/typescript/package.json +++ b/sdks/typescript/package.json @@ -10,7 +10,7 @@ "eslint": "^8.15.0", "istanbul": "^0.4.5", "js-yaml": "^4.1.0", - "mocha": "^9.1.3", + "mocha": "^11.1.0", "prettier": "^3.1.1", "typedoc": "^0.23.23", "typescript": "4.7" From 667b0457e3360e84b3c7d6bb2c2f7d96f91ad4ed Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 19 Mar 2025 10:46:04 -0700 Subject: [PATCH 36/48] Bump github.com/fsouza/fake-gcs-server from 1.52.1 to 1.52.2 in /sdks (#34094) Bumps [github.com/fsouza/fake-gcs-server](https://github.com/fsouza/fake-gcs-server) from 1.52.1 to 1.52.2. - [Release notes](https://github.com/fsouza/fake-gcs-server/releases) - [Commits](https://github.com/fsouza/fake-gcs-server/compare/v1.52.1...v1.52.2) --- updated-dependencies: - dependency-name: github.com/fsouza/fake-gcs-server 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 | 14 ++++++++------ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 013d60904913..6f9525acdcf6 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -70,7 +70,7 @@ require ( require ( github.com/avast/retry-go/v4 v4.6.0 - github.com/fsouza/fake-gcs-server v1.52.1 + github.com/fsouza/fake-gcs-server v1.52.2 github.com/golang-cz/devslog v0.0.11 golang.org/x/exp v0.0.0-20250106191152-7588d65b2ba8 ) @@ -159,7 +159,7 @@ require ( github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.2.1 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/goccy/go-json v0.10.4 // indirect + github.com/goccy/go-json v0.10.5 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8 // indirect github.com/golang/snappy v0.0.4 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 509d325e71a6..3318aabdcefb 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -924,8 +924,8 @@ github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzP github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= -github.com/fsouza/fake-gcs-server v1.52.1 h1:Hx3G2ZpyBzHGmW7cHURWWoTm6jM3M5fcWMIAHBYlJyc= -github.com/fsouza/fake-gcs-server v1.52.1/go.mod h1:Paxf25VmSNMN52L+2/cVulF5fkLUA0YJIYjTGJiwf3c= +github.com/fsouza/fake-gcs-server v1.52.2 h1:j6ne83nqHrlX5EEor7WWVIKdBsztGtwJ1J2mL+k+iio= +github.com/fsouza/fake-gcs-server v1.52.2/go.mod h1:47HKyIkz6oLTes1R8vEaHLwXfzYsGfmDUk1ViHHAUsA= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= github.com/gin-gonic/gin v1.6.3/go.mod h1:75u5sXoLsGZoRN5Sgbi1eraJ4GU3++wFwWzhwvtwp4M= @@ -971,8 +971,8 @@ github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22 github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/goccy/go-json v0.10.4 h1:JSwxQzIqKfmFX1swYPpUThQZp/Ka4wzJdK0LWVytLPM= -github.com/goccy/go-json v0.10.4/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= +github.com/goccy/go-json v0.10.5 h1:Fq85nIqj+gXn/S5ahsiTlK3TmC85qgirsdTP/+DeaC4= +github.com/goccy/go-json v0.10.5/go.mod h1:oq7eo15ShAhp70Anwd5lgX2pLfOS3QCiwU/PULtXL6M= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gofrs/uuid v4.0.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= @@ -1269,13 +1269,15 @@ github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/ github.com/mattn/go-sqlite3 v1.14.14/go.mod h1:NyWgC/yNuGj7Q9rpYnZvas74GogHl5/Z4A/KQRfk6bU= github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY= github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= +github.com/minio/crc64nvme v1.0.0 h1:MeLcBkCTD4pAoU7TciAfwsfxgkhM2u5hCe48hSEVFr0= +github.com/minio/crc64nvme v1.0.0/go.mod h1:eVfm2fAzLlxMdUGc0EEBGSMmPwmXD5XiNRpnu9J3bvg= github.com/minio/highwayhash v1.0.3 h1:kbnuUMoHYyVl7szWjSxJnxw11k2U709jqFPPmIUyD6Q= github.com/minio/highwayhash v1.0.3/go.mod h1:GGYsuwP/fPD6Y9hMiXuapVvlIUEhFhMTh0rxU3ik1LQ= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.34/go.mod h1:nCrRzjoSUQh8hgKKtu3Y708OLvRLtuASMg2/nvmbarw= -github.com/minio/minio-go/v7 v7.0.83 h1:W4Kokksvlz3OKf3OqIlzDNKd4MERlC2oN8YptwJ0+GA= -github.com/minio/minio-go/v7 v7.0.83/go.mod h1:57YXpvc5l3rjPdhqNrDsvVlY0qPI6UTk1bflAe+9doY= +github.com/minio/minio-go/v7 v7.0.86 h1:DcgQ0AUjLJzRH6y/HrxiZ8CXarA70PAIufXHodP4s+k= +github.com/minio/minio-go/v7 v7.0.86/go.mod h1:VbfO4hYwUu3Of9WqGLBZ8vl3Hxnxo4ngxK4hzQDf4x4= github.com/minio/sha256-simd v1.0.0/go.mod h1:OuYzVNI5vcoYIAmbIvHPl3N3jUzVedXbKy5RFepssQM= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.3.3/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= From 457fe82446432d070bd5a03703c7e75503aa8867 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 19 Mar 2025 10:47:53 -0700 Subject: [PATCH 37/48] Bump golang.org/x/net from 0.23.0 to 0.36.0 in /learning/katas/go (#34267) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.23.0 to 0.36.0. - [Commits](https://github.com/golang/net/compare/v0.23.0...v0.36.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- learning/katas/go/go.mod | 4 ++-- learning/katas/go/go.sum | 38 ++++++++++++++++++++++++++++++-------- 2 files changed, 32 insertions(+), 10 deletions(-) diff --git a/learning/katas/go/go.mod b/learning/katas/go/go.mod index 089340e0aa43..69e347885d33 100644 --- a/learning/katas/go/go.mod +++ b/learning/katas/go/go.mod @@ -19,7 +19,7 @@ go 1.14 require ( github.com/apache/beam/sdks/v2 v2.40.0 - github.com/google/go-cmp v0.5.8 - golang.org/x/net v0.23.0 // indirect + github.com/google/go-cmp v0.6.0 + golang.org/x/net v0.36.0 // indirect google.golang.org/genproto v0.0.0-20220815135757-37a418bb8959 // indirect ) diff --git a/learning/katas/go/go.sum b/learning/katas/go/go.sum index 155ba0cbdab9..eaf579e16cb5 100644 --- a/learning/katas/go/go.sum +++ b/learning/katas/go/go.sum @@ -427,8 +427,9 @@ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.7/go.mod h1:n+brtR0CgQNWTVd5ZUFpTBC8YFBDLK/h/bpaJ8/DtOE= -github.com/google/go-cmp v0.5.8 h1:e6P7q2lk1O+qJJb4BtCQXlK8vWEO8V1ZeuEdJNOqZyg= github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/gofuzz v1.1.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= @@ -781,8 +782,10 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= 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.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= -golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= +golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= +golang.org/x/crypto v0.35.0/go.mod h1:dy7dXNW32cAb/6/PRuTNsix8T+vJAqvuIy5Bli/x0YQ= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -820,6 +823,9 @@ golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -879,9 +885,11 @@ golang.org/x/net v0.0.0-20220607020251-c690dde0001d/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= -golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= -golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= 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= @@ -918,6 +926,10 @@ golang.org/x/sync v0.0.0-20220513210516-0976fa681c29/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220601150217-0de741cfad7f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1020,15 +1032,20 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= -golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= 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.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= -golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= +golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= +golang.org/x/term v0.29.0/go.mod h1:6bl4lRlvVuDgSf3179VpIxBF0o10JUpXWOnI7nErv7s= 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= @@ -1040,8 +1057,11 @@ golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1108,6 +1128,8 @@ golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= From 6c156c5499c276dd922d03193ffa1dbedece073a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 19 Mar 2025 13:41:59 -0700 Subject: [PATCH 38/48] Fix pandas doctests sensitive to NumpyExtensionArray formatting. (#34336) --- sdks/python/apache_beam/dataframe/doctests.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sdks/python/apache_beam/dataframe/doctests.py b/sdks/python/apache_beam/dataframe/doctests.py index 57ee8009ba44..7d2e98bfe14c 100644 --- a/sdks/python/apache_beam/dataframe/doctests.py +++ b/sdks/python/apache_beam/dataframe/doctests.py @@ -288,6 +288,12 @@ def sort_and_normalize(text): want = sort_and_normalize(want) except Exception: got = traceback.format_exc() + + if sys.version_info < (3, 11) and 'NumpyExtensionArray' in got: + # Work around formatting differences (np.int32(1) vs 1). + got = re.sub('np.(int32|str_)[(]([^()]+)[)]', r'\2', got) + got = re.sub('np.complex128[(]([^()]+)[)]', r'(\1)', got) + return want, got @property From 5ad150b19ed64c5c2b4d900b3f2e3fcfe1e8019d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 19 Mar 2025 14:11:17 -0700 Subject: [PATCH 39/48] Bump google.golang.org/api from 0.221.0 to 0.226.0 in /sdks (#34293) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.221.0 to 0.226.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.221.0...v0.226.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 | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 6f9525acdcf6..ca744024dd78 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -60,7 +60,7 @@ require ( golang.org/x/sync v0.12.0 golang.org/x/sys v0.31.0 golang.org/x/text v0.23.0 - google.golang.org/api v0.224.0 + google.golang.org/api v0.226.0 google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb google.golang.org/grpc v1.71.0 google.golang.org/protobuf v1.36.5 @@ -125,7 +125,7 @@ require ( go.opentelemetry.io/otel/sdk/metric v1.34.0 // indirect go.opentelemetry.io/otel/trace v1.34.0 // indirect go.shabbyrobe.org/gocovmerge v0.0.0-20230507111327-fa4f82cfbf4d // indirect - golang.org/x/time v0.10.0 // indirect + golang.org/x/time v0.11.0 // indirect ) require ( diff --git a/sdks/go.sum b/sdks/go.sum index 3318aabdcefb..070138a3e126 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1865,8 +1865,8 @@ golang.org/x/time v0.0.0-20220224211638-0e9765cccd65/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20220922220347-f3bd1da661af/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.1.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.10.0 h1:3usCWA8tQn0L8+hFJQNgzpWbd89begxN66o1Ojdn5L4= -golang.org/x/time v0.10.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.11.0 h1:/bpjEDfN9tkoN/ryeYHnv5hcMlc8ncjMcM4XBk5NWV0= +golang.org/x/time v0.11.0/go.mod h1:CDIdPxbZBQxdj6cxyCIdrNogrJKMJ7pr37NYpMcMDSg= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -2031,8 +2031,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.224.0 h1:Ir4UPtDsNiwIOHdExr3fAj4xZ42QjK7uQte3lORLJwU= -google.golang.org/api v0.224.0/go.mod h1:3V39my2xAGkodXy0vEqcEtkqgw2GtrFL5WuBZlCTCOQ= +google.golang.org/api v0.226.0 h1:9A29y1XUD+YRXfnHkO66KggxHBZWg9LsTGqm7TkUvtQ= +google.golang.org/api v0.226.0/go.mod h1:WP/0Xm4LVvMOCldfvOISnWquSRWbG2kArDZcg+W2DbY= 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= From 65fab8cac9b5479f98ba78ffaebcd6118830111a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 19 Mar 2025 12:47:53 -0700 Subject: [PATCH 40/48] Fix relative path yaml provider error when no base provided. --- sdks/python/apache_beam/yaml/yaml_provider.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index eca6c5f46dd2..9e30c8ccae07 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -1372,6 +1372,8 @@ def wrapper(*args, **kwargs): def _join_url_or_filepath(base, path): + if not base: + return path base_scheme = urllib.parse.urlparse(base, '').scheme path_scheme = urllib.parse.urlparse(path, base_scheme).scheme if path_scheme != base_scheme: From ded696c6f8272b1cc13adfcb43d0856b202f1ce5 Mon Sep 17 00:00:00 2001 From: Talat Uyarer Date: Thu, 20 Mar 2025 11:21:59 -0700 Subject: [PATCH 41/48] Fixed infinitive loop --- .../sql/meta/provider/kafka/KafkaTableProvider.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java index fb2fab4c8562..45e99e05f584 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProvider.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.auto.service.AutoService; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map.Entry; import java.util.Optional; @@ -145,8 +146,9 @@ public BeamSqlTable buildBeamSqlTable(Table table) { // Get Consumer Properties from Table properties HashMap configUpdates = new HashMap(); - while (properties.fields().hasNext()) { - Entry field = properties.fields().next(); + Iterator> tableProperties = properties.fields(); + while (tableProperties.hasNext()) { + Entry field = tableProperties.next(); if (field.getKey().startsWith("properties.")) { configUpdates.put(field.getKey().replace("properties.", ""), field.getValue().textValue()); } From c73226a650021cd4bddc32c8553a0c4027583a51 Mon Sep 17 00:00:00 2001 From: scwhittle Date: Thu, 20 Mar 2025 10:36:27 +0100 Subject: [PATCH 42/48] Avoid holding data elements alive via stack frame gc roots. (#33086) This is accomplished by changing to iterators which throw away elements that have been passed as well as just referring to the inputstream of the element instead of the entire stream. If the ByteString is based upon a ByteBuffer this will allow blocks that have been advanced past to be gc'd. --- .../fn/data/BeamFnDataInboundObserver.java | 100 ++++++++++++++---- .../harness/control/ProcessBundleHandler.java | 4 +- 2 files changed, 83 insertions(+), 21 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java index dab497bd3128..ee4a36f0171a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java @@ -19,15 +19,19 @@ import java.io.InputStream; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.CancellableQueue; +import org.checkerframework.checker.nullness.qual.Nullable; /** * Decodes {@link BeamFnApi.Elements} partitioning them using the provided {@link DataEndpoint}s and @@ -79,7 +83,7 @@ private CloseException() { private final int totalNumEndpoints; private int numEndpointsThatAreIncomplete; - private AtomicBoolean consumingReceivedData; + private final AtomicBoolean consumingReceivedData; private BeamFnDataInboundObserver( List> dataEndpoints, List> timerEndpoints) { @@ -118,6 +122,41 @@ public boolean isConsumingReceivedData() { return consumingReceivedData.get(); } + // Copies the elements of list to an array and removes references to elements that + // have been iterated past. + private static class DiscardingIterator implements Iterator { + private int index = 0; + private final @Nullable Object[] array; + + DiscardingIterator(List list) { + this.array = list.toArray(); + } + + @Override + public boolean hasNext() { + return index < array.length; + } + + @Override + public T next() { + if (index >= array.length) { + throw new NoSuchElementException(); + } + @SuppressWarnings("unchecked") + T result = (T) array[index]; + array[index] = null; + ++index; + return result; + } + } + + private static Iterator createDiscardingIterator(List list) { + if (list.isEmpty()) { + return Collections.emptyIterator(); // Optimize empty lists, which are common for timers. + } + return new DiscardingIterator<>(list); + } + /** * Uses the callers thread to process all elements received until we receive the end of the stream * from the upstream producer for all endpoints specified. @@ -130,10 +169,20 @@ public void awaitCompletion() throws Exception { // The SDK indicates it has consumed all the received data before it attempts to take // more elements off the queue. consumingReceivedData.set(false); - BeamFnApi.Elements elements = queue.take(); + + // We use discarding iterators and don't reference the elements longer than necessary to + // avoid pinning possibly large data pages and making them ineligible for garbage + // collection. + Iterator dataIterator; + Iterator timersIterator; + { + BeamFnApi.Elements elements = queue.take(); + dataIterator = createDiscardingIterator(elements.getDataList()); + timersIterator = createDiscardingIterator(elements.getTimersList()); + } // The SDK is no longer blocked on receiving more data from the Runner. consumingReceivedData.set(true); - if (multiplexElements(elements)) { + if (multiplexElements(dataIterator, timersIterator)) { return; } } @@ -150,34 +199,45 @@ public void awaitCompletion() throws Exception { * Dispatches the data and timers from the elements to corresponding receivers. Returns true if * all the endpoints are done after elements dispatching. */ - public boolean multiplexElements(Elements elements) throws Exception { - for (BeamFnApi.Elements.Data data : elements.getDataList()) { - EndpointStatus> endpoint = - transformIdToDataEndpoint.get(data.getTransformId()); - if (endpoint == null) { - throw new IllegalStateException( - String.format( - "Unable to find inbound data receiver for instruction %s and transform %s.", - data.getInstructionId(), data.getTransformId())); - } else if (endpoint.isDone) { - throw new IllegalStateException( - String.format( - "Received data after inbound data receiver is done for instruction %s and transform %s.", - data.getInstructionId(), data.getTransformId())); + public boolean multiplexElements( + Iterator dataElements, Iterator timerElements) + throws Exception { + while (dataElements.hasNext()) { + // We're careful to avoid references to the full data while processing, allowing the input + // stream to possibly cleanup memory as it advances. + InputStream inputStream; + EndpointStatus> endpoint; + boolean isLast; + { + Elements.Data data = dataElements.next(); + isLast = data.getIsLast(); + endpoint = transformIdToDataEndpoint.get(data.getTransformId()); + if (endpoint == null) { + throw new IllegalStateException( + String.format( + "Unable to find inbound data receiver for instruction %s and transform %s.", + data.getInstructionId(), data.getTransformId())); + } else if (endpoint.isDone) { + throw new IllegalStateException( + String.format( + "Received data after inbound data receiver is done for instruction %s and transform %s.", + data.getInstructionId(), data.getTransformId())); + } + inputStream = data.getData().newInput(); } - InputStream inputStream = data.getData().newInput(); Coder coder = (Coder) endpoint.endpoint.getCoder(); FnDataReceiver receiver = (FnDataReceiver) endpoint.endpoint.getReceiver(); while (inputStream.available() > 0) { receiver.accept(coder.decode(inputStream)); } - if (data.getIsLast()) { + if (isLast) { endpoint.isDone = true; numEndpointsThatAreIncomplete -= 1; } } - for (BeamFnApi.Elements.Timers timers : elements.getTimersList()) { + while (timerElements.hasNext()) { + Elements.Timers timers = timerElements.next(); Map>> timerFamilyIdToEndpoints = transformIdToTimerFamilyIdToTimerEndpoint.get(timers.getTransformId()); if (timerFamilyIdToEndpoints == null) { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 12441b0ea6ea..90e34a89a021 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -525,7 +525,9 @@ public BeamFnApi.InstructionResponse.Builder processBundle(InstructionRequest re boolean inputFinished = bundleProcessor .getInboundObserver() - .multiplexElements(request.getProcessBundle().getElements()); + .multiplexElements( + request.getProcessBundle().getElements().getDataList().iterator(), + request.getProcessBundle().getElements().getTimersList().iterator()); if (!inputFinished) { throw new RuntimeException( "Elements embedded in ProcessBundleRequest do not contain stream terminators for " From 3bd8e025673b785db1b3d7b10f7925e756adfd62 Mon Sep 17 00:00:00 2001 From: akashorabek <70029317+akashorabek@users.noreply.github.com> Date: Thu, 20 Mar 2025 18:56:11 +0500 Subject: [PATCH 43/48] Remove finalizers from sdb-cluster before deleting singlestoreio namespaces (#34359) --- .test-infra/tools/stale_k8s_workload_cleaner.sh | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/.test-infra/tools/stale_k8s_workload_cleaner.sh b/.test-infra/tools/stale_k8s_workload_cleaner.sh index c167871c8b94..dc6cadcf0c4f 100755 --- a/.test-infra/tools/stale_k8s_workload_cleaner.sh +++ b/.test-infra/tools/stale_k8s_workload_cleaner.sh @@ -25,6 +25,7 @@ set -euo pipefail PROJECT=apache-beam-testing LOCATION=us-central1-a CLUSTER=io-datastores +MEMSQL_CLUSTER_RESOURCE="memsqlclusters.memsql.com/sdb-cluster" function should_teardown() { if [[ $1 =~ ^([0-9]+)([a-z]) ]]; then @@ -43,10 +44,12 @@ function should_teardown() { gcloud container clusters get-credentials io-datastores --zone us-central1-a --project apache-beam-testing while read NAME STATUS AGE; do - # Regex has temporary workaround to avoid trying to delete beam-performancetests-singlestoreio-* to avoid getting stuck in a terminal state - # See https://github.com/apache/beam/pull/33545 for context. - # This may be safe to remove if https://cloud.google.com/knowledge/kb/deleted-namespace-remains-in-terminating-status-000004867 has been resolved, just try it before checking in :) - if [[ $NAME =~ ^beam-.+(test|-it)(?!s-singlestoreio) ]] && should_teardown $AGE; then + if [[ $NAME =~ ^beam-.+(test|-it) ]] && should_teardown $AGE; then + # For namespaces containing "-singlestoreio-", remove the finalizers from the sdb-cluster resource + # to ensure it can be fully deleted and not block namespace removal. + if [[ $NAME == *-singlestoreio-* ]]; then + kubectl patch $MEMSQL_CLUSTER_RESOURCE -n $NAME -p '[{"op": "remove", "path": "/metadata/finalizers"}]' --type=json + fi kubectl delete namespace $NAME fi done < <( kubectl get namespaces --context=gke_${PROJECT}_${LOCATION}_${CLUSTER} ) From 9864fd694d8254c0212060922c2b06323bac2c34 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 20 Mar 2025 10:51:57 -0400 Subject: [PATCH 44/48] Bump google.golang.org/api from 0.226.0 to 0.227.0 in /sdks (#34356) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.226.0 to 0.227.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.226.0...v0.227.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 | 6 +++--- sdks/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index ca744024dd78..c60a2a0dac31 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -60,7 +60,7 @@ require ( golang.org/x/sync v0.12.0 golang.org/x/sys v0.31.0 golang.org/x/text v0.23.0 - google.golang.org/api v0.226.0 + google.golang.org/api v0.227.0 google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb google.golang.org/grpc v1.71.0 google.golang.org/protobuf v1.36.5 @@ -167,7 +167,7 @@ require ( github.com/google/pprof v0.0.0-20241210010833-40e02aabc2ad // indirect github.com/google/renameio/v2 v2.0.0 // indirect github.com/google/s2a-go v0.1.9 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.3.5 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.3.6 // indirect github.com/googleapis/gax-go/v2 v2.14.1 // indirect github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect @@ -199,7 +199,7 @@ require ( golang.org/x/tools v0.29.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect google.golang.org/genproto/googleapis/api v0.0.0-20250303144028-a0af3efb3deb // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20250303144028-a0af3efb3deb // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250313205543-e70fdf4c4cb4 // indirect ) // Hopefully a temporary measure to sort out a mismatch between grpc packages diff --git a/sdks/go.sum b/sdks/go.sum index 070138a3e126..e23524913580 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1110,8 +1110,8 @@ github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY github.com/googleapis/enterprise-certificate-proxy v0.2.0/go.mod h1:8C0jb7/mgJe/9KK8Lm7X9ctZC2t60YyIpYEI16jx0Qg= github.com/googleapis/enterprise-certificate-proxy v0.2.1/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= -github.com/googleapis/enterprise-certificate-proxy v0.3.5 h1:VgzTY2jogw3xt39CusEnFJWm7rlsq5yL5q9XdLOuP5g= -github.com/googleapis/enterprise-certificate-proxy v0.3.5/go.mod h1:MkHOF77EYAE7qfSuSS9PU6g4Nt4e11cnsDUowfwewLA= +github.com/googleapis/enterprise-certificate-proxy v0.3.6 h1:GW/XbdyBFQ8Qe+YAmFU9uHLo7OnF5tL52HFAgMmyrf4= +github.com/googleapis/enterprise-certificate-proxy v0.3.6/go.mod h1:MkHOF77EYAE7qfSuSS9PU6g4Nt4e11cnsDUowfwewLA= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pfu6SK+H1/DsU0= @@ -2031,8 +2031,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.226.0 h1:9A29y1XUD+YRXfnHkO66KggxHBZWg9LsTGqm7TkUvtQ= -google.golang.org/api v0.226.0/go.mod h1:WP/0Xm4LVvMOCldfvOISnWquSRWbG2kArDZcg+W2DbY= +google.golang.org/api v0.227.0 h1:QvIHF9IuyG6d6ReE+BNd11kIB8hZvjN8Z5xY5t21zYc= +google.golang.org/api v0.227.0/go.mod h1:EIpaG6MbTgQarWF5xJvX0eOJPK9n/5D4Bynb9j2HXvQ= 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= @@ -2195,8 +2195,8 @@ google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb h1:ITgPrl429bc6+2Z google.golang.org/genproto v0.0.0-20250303144028-a0af3efb3deb/go.mod h1:sAo5UzpjUwgFBCzupwhcLcxHVDK7vG5IqI30YnwX2eE= google.golang.org/genproto/googleapis/api v0.0.0-20250303144028-a0af3efb3deb h1:p31xT4yrYrSM/G4Sn2+TNUkVhFCbG9y8itM2S6Th950= google.golang.org/genproto/googleapis/api v0.0.0-20250303144028-a0af3efb3deb/go.mod h1:jbe3Bkdp+Dh2IrslsFCklNhweNTBgSYanP1UXhJDhKg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250303144028-a0af3efb3deb h1:TLPQVbx1GJ8VKZxz52VAxl1EBgKXXbTiU9Fc5fZeLn4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250303144028-a0af3efb3deb/go.mod h1:LuRYeWDFV6WOn90g357N17oMCaxpgCnbi/44qJvDn2I= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250313205543-e70fdf4c4cb4 h1:iK2jbkWL86DXjEx0qiHcRE9dE4/Ahua5k6V8OWFb//c= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250313205543-e70fdf4c4cb4/go.mod h1:LuRYeWDFV6WOn90g357N17oMCaxpgCnbi/44qJvDn2I= 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 0e0bc35a718cb4e1aab24a6b57aa2434ed641c51 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 20 Mar 2025 10:52:26 -0400 Subject: [PATCH 45/48] Bump github.com/nats-io/nats-server/v2 from 2.10.25 to 2.11.0 in /sdks (#34358) Bumps [github.com/nats-io/nats-server/v2](https://github.com/nats-io/nats-server) from 2.10.25 to 2.11.0. - [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.25...v2.11.0) --- updated-dependencies: - dependency-name: github.com/nats-io/nats-server/v2 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 | 7 ++++--- sdks/go.sum | 16 ++++++++++------ 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index c60a2a0dac31..cd3d34595bc8 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -46,7 +46,7 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20250106100439-5c39aecd6999 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.13.1 - github.com/nats-io/nats-server/v2 v2.10.25 + github.com/nats-io/nats-server/v2 v2.11.0 github.com/nats-io/nats.go v1.39.1 github.com/proullon/ramsql v0.1.4 github.com/spf13/cobra v1.9.1 @@ -96,13 +96,14 @@ require ( github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect + github.com/google/go-tpm v0.9.3 // indirect github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 // indirect github.com/minio/highwayhash v1.0.3 // indirect github.com/moby/docker-image-spec v1.3.1 // indirect github.com/moby/sys/user v0.3.0 // indirect github.com/moby/sys/userns v0.1.0 // indirect github.com/nats-io/jwt/v2 v2.7.3 // indirect - github.com/nats-io/nkeys v0.4.9 // indirect + github.com/nats-io/nkeys v0.4.10 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -172,7 +173,7 @@ require ( github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect - github.com/klauspost/compress v1.17.11 // indirect + github.com/klauspost/compress v1.18.0 // indirect github.com/klauspost/cpuid/v2 v2.2.9 // indirect github.com/magiconair/properties v1.8.9 // indirect github.com/moby/patternmatcher v0.6.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index e23524913580..5eb99336a788 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -722,6 +722,8 @@ github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3 github.com/ajstarks/svgo v0.0.0-20211024235047-1546f124cd8b/go.mod h1:1KcenG0jGWcpt8ov532z81sp/kMMUG485J2InIOyADM= github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/antithesishq/antithesis-sdk-go v0.4.3-default-no-op h1:+OSa/t11TFhqfrX0EOSqQBDJ0YlpmK0rDSiB19dg9M0= +github.com/antithesishq/antithesis-sdk-go v0.4.3-default-no-op/go.mod h1:IUpT2DPAKh6i/YhSbt6Gl3v2yvUZjmKncl7U91fup7E= github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0= github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40 h1:q4dksr6ICHXqG5hm0ZW5IHyeEJXoIJSOZeBLmWPNeIQ= github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40/go.mod h1:Q7yQnSMnLvcXlZ8RV+jwz/6y1rQTqbX6C82SndT52Zs= @@ -1063,6 +1065,8 @@ github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= github.com/google/go-replayers/grpcreplay v1.1.0/go.mod h1:qzAvJ8/wi57zq7gWqaE6AwLM6miiXUQwP1S+I9icmhk= github.com/google/go-replayers/httpreplay v1.1.1/go.mod h1:gN9GeLIs7l6NUoVaSSnv2RiqK1NiwAmD0MrKeC9IIks= +github.com/google/go-tpm v0.9.3 h1:+yx0/anQuGzi+ssRqeD6WpXjW2L/V0dItUayO0i9sRc= +github.com/google/go-tpm v0.9.3/go.mod h1:h9jEsEECg7gtLis0upRBQU+GhYVH6jMjrFxI8u6bVUY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian v2.1.1-0.20190517191504-25dcb96d9e51+incompatible h1:xmapqc1AyLoB+ddYT6r04bD9lIjlOqGaREovi0SzFaE= @@ -1219,8 +1223,8 @@ github.com/klauspost/compress v1.10.3/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYs github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.15.1/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.17.11 h1:In6xLpyWOi1+C7tXUUWv2ot1QvBjxevKAaI6IXrJmUc= -github.com/klauspost/compress v1.17.11/go.mod h1:pMDklpSncoRMuLFrf1W9Ss9KT+0rH90U12bZKk7uwG0= +github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= +github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= github.com/klauspost/cpuid/v2 v2.0.1/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.0.4/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= github.com/klauspost/cpuid/v2 v2.0.9/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa0213Md/qVLRg= @@ -1307,12 +1311,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.7.3 h1:6bNPK+FXgBeAqdj4cYQ0F8ViHRbi7woQLq4W29nUAzE= github.com/nats-io/jwt/v2 v2.7.3/go.mod h1:GvkcbHhKquj3pkioy5put1wvPxs78UlZ7D/pY+BgZk4= -github.com/nats-io/nats-server/v2 v2.10.25 h1:J0GWLDDXo5HId7ti/lTmBfs+lzhmu8RPkoKl0eSCqwc= -github.com/nats-io/nats-server/v2 v2.10.25/go.mod h1:/YYYQO7cuoOBt+A7/8cVjuhWTaTUEAlZbJT+3sMAfFU= +github.com/nats-io/nats-server/v2 v2.11.0 h1:fdwAT1d6DZW/4LUz5rkvQUe5leGEwjjOQYntzVRKvjE= +github.com/nats-io/nats-server/v2 v2.11.0/go.mod h1:leXySghbdtXSUmWem8K9McnJ6xbJOb0t9+NQ5HTRZjI= github.com/nats-io/nats.go v1.39.1 h1:oTkfKBmz7W047vRxV762M67ZdXeOtUgvbBaNoQ+3PPk= github.com/nats-io/nats.go v1.39.1/go.mod h1:MgRb8oOdigA6cYpEPhXJuRVH6UE/V4jblJ2jQ27IXYM= -github.com/nats-io/nkeys v0.4.9 h1:qe9Faq2Gxwi6RZnZMXfmGMZkg3afLLOtrU+gDZJ35b0= -github.com/nats-io/nkeys v0.4.9/go.mod h1:jcMqs+FLG+W5YO36OX6wFIFcmpdAns+w1Wm6D3I/evE= +github.com/nats-io/nkeys v0.4.10 h1:glmRrpCmYLHByYcePvnTBEAwawwapjCPMjy2huw20wc= +github.com/nats-io/nkeys v0.4.10/go.mod h1:OjRrnIKnWBFl+s4YK5ChQfvHP2fxqZexrKJoVVyWB3U= 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 a90e411a7c6c41393425bd323c32940b01ea5494 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Thu, 20 Mar 2025 15:46:09 +0000 Subject: [PATCH 46/48] [IcebergIO] Filter out data files that have already been committed (#34264) * remove already committed files * changes * simplify validation * validate without loading the whole collection into memory --- CHANGES.md | 1 + .../sdk/io/iceberg/AppendFilesToTables.java | 35 +++++++++++++++++-- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 5e19cfbe617e..7818916cf93f 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -66,6 +66,7 @@ * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * [Java] Use API compatible with both com.google.cloud.bigdataoss:util 2.x and 3.x in BatchLoads ([#34105](https://github.com/apache/beam/pull/34105)) +* [IcebergIO] Address edge case where bundle retry following a successful data commit results in data duplication ([#34264](https://github.com/apache/beam/pull/34264)) ## New Features / Improvements diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java index fed72a381d5e..024e0336e67d 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java @@ -128,13 +128,12 @@ public void processElement( BoundedWindow window) throws IOException { String tableStringIdentifier = element.getKey(); + Table table = getCatalog().loadTable(TableIdentifier.parse(element.getKey())); Iterable fileWriteResults = element.getValue(); - if (!fileWriteResults.iterator().hasNext()) { + if (shouldSkip(table, fileWriteResults)) { return; } - Table table = getCatalog().loadTable(TableIdentifier.parse(element.getKey())); - // vast majority of the time, we will simply append data files. // in the rare case we get a batch that contains multiple partition specs, we will group // data into manifest files and append. @@ -211,5 +210,35 @@ private ManifestWriter createManifestWriter( tableLocation, manifestFilePrefix, uuid, spec.specId())); return ManifestFiles.write(spec, io.newOutputFile(location)); } + + // If the process call fails immediately after a successful commit, it gets retried with + // the same data, possibly leading to data duplication. + // To mitigate, we skip the current batch of files if it matches the most recently committed + // batch. + // + // TODO(ahmedabu98): This does not cover concurrent writes from other pipelines, where the + // "last successful snapshot" might reflect commits from other sources. Ideally, we would make + // this stateful, but that is update incompatible. + // TODO(ahmedabu98): add load test pipelines with intentional periodic crashing + private boolean shouldSkip(Table table, Iterable fileWriteResults) { + if (table.currentSnapshot() == null) { + return false; + } + if (!fileWriteResults.iterator().hasNext()) { + return true; + } + + // Check if the current batch is identical to the most recently committed batch. + // Upstream GBK means we always get the same batch of files on retry, + // so a single overlapping file means the whole batch is identical. + String sampleCommittedDataFilePath = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().path().toString(); + for (FileWriteResult result : fileWriteResults) { + if (result.getSerializableDataFile().getPath().equals(sampleCommittedDataFilePath)) { + return true; + } + } + return false; + } } } From b7d6e8886cbeddebdc62d78804ce0fd310c81fb2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rados=C5=82aw=20Stankiewicz?= Date: Thu, 20 Mar 2025 17:30:34 +0100 Subject: [PATCH 47/48] [Python] File staging to user worker support (#34208) * file staging * add tests * help message * format * typo * yapf * docs * support for multiple files * changes.md * fix test --- CHANGES.md | 2 +- .../apache_beam/options/pipeline_options.py | 13 +++++++--- .../apache_beam/runners/portability/stager.py | 6 +++++ .../runners/portability/stager_test.py | 24 +++++++++++++++++++ 4 files changed, 41 insertions(+), 4 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 7818916cf93f..92e22e6eec60 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -139,7 +139,7 @@ * [Dataflow Streaming] Enable Windmill GetWork Response Batching by default ([#33847](https://github.com/apache/beam/pull/33847)). * With this change user workers will request batched GetWork responses from backend and backend will send multiple WorkItems in the same response proto. * The feature can be disabled by passing `--windmillRequestBatchedGetWorkResponse=false` - +* Added supports for staging arbitrary files via `--files_to_stage` flag (Python) ([#34208](https://github.com/apache/beam/pull/34208)) ## Breaking Changes * AWS V1 I/Os have been removed (Java). As part of this, x-lang Python Kinesis I/O has been updated to consume the V2 IO and it also no longer supports setting producer_properties ([#33430](https://github.com/apache/beam/issues/33430)). diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index 6de15b90790d..2ddce625b210 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -1568,6 +1568,16 @@ def _add_argparse_args(cls, parser): 'staged in the staging area (--staging_location option) and the ' 'workers will install them in same order they were specified on ' 'the command line.')) + parser.add_argument( + '--files_to_stage', + dest='files_to_stage', + action='append', + default=None, + help=( + 'Local path to a file. During job submission, the files will be ' + 'staged in the staging area (--staging_location option) and then ' + 'workers will upload them to the worker specific staging location ' + '(e.g. $SEMI_PERSISTENT_DIRECTORY/staged/ for portable runner.')) parser.add_argument( '--prebuild_sdk_container_engine', help=( @@ -1899,9 +1909,6 @@ def _add_argparse_args(cls, parser): ) -# TODO(silviuc): Add --files_to_stage option. -# This could potentially replace the --requirements_file and --setup_file. - # TODO(silviuc): Non-standard options. Keep them? If yes, add help too! # Remote execution must check that this option is not None. diff --git a/sdks/python/apache_beam/runners/portability/stager.py b/sdks/python/apache_beam/runners/portability/stager.py index c7142bfddcaf..cb0a3c88a477 100644 --- a/sdks/python/apache_beam/runners/portability/stager.py +++ b/sdks/python/apache_beam/runners/portability/stager.py @@ -292,6 +292,12 @@ def create_job_resources( Stager._create_file_stage_to_artifact( tarball_file, WORKFLOW_TARBALL_FILE)) + if setup_options.files_to_stage is not None: + for file in setup_options.files_to_stage: + resources.append( + Stager._create_file_stage_to_artifact( + file, os.path.basename(file))) + # Handle extra local packages that should be staged. if setup_options.extra_packages is not None: resources.extend( diff --git a/sdks/python/apache_beam/runners/portability/stager_test.py b/sdks/python/apache_beam/runners/portability/stager_test.py index 5535989a5786..85443c0a1b5f 100644 --- a/sdks/python/apache_beam/runners/portability/stager_test.py +++ b/sdks/python/apache_beam/runners/portability/stager_test.py @@ -428,6 +428,30 @@ def test_sdk_location_local_directory(self): with open(tarball_path) as f: self.assertEqual(f.read(), 'Package content.') + def test_files_to_stage(self): + staging_dir = self.make_temp_dir() + source_dir = self.make_temp_dir() + + foo_ca = os.path.join(source_dir, 'foo.ca') + self.create_temp_file(foo_ca, 'ca content') + test_txt = os.path.join(source_dir, 'test.txt') + self.create_temp_file(test_txt, 'test content') + files_to_stage = [foo_ca, test_txt] + options = PipelineOptions() + self.update_options(options) + options.view_as(SetupOptions).files_to_stage = files_to_stage + + self.assertEqual( + ['foo.ca', 'test.txt', stager.SUBMISSION_ENV_DEPENDENCIES_FILE], + self.stager.create_and_stage_job_resources( + options, staging_location=staging_dir)[1]) + foo = os.path.join(staging_dir, 'foo.ca') + with open(foo) as f: + self.assertEqual(f.read(), 'ca content') + txt = os.path.join(staging_dir, 'test.txt') + with open(txt) as f: + self.assertEqual(f.read(), 'test content') + def test_sdk_location_local_source_file(self): staging_dir = self.make_temp_dir() sdk_directory = self.make_temp_dir() From e96dff4cfed4903b061b61f67fbbd2389298903d Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 20 Mar 2025 13:02:42 -0400 Subject: [PATCH 48/48] Fix dockerfile version (#34352) --- examples/notebooks/beam-ml/run_inference_vllm.ipynb | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/notebooks/beam-ml/run_inference_vllm.ipynb b/examples/notebooks/beam-ml/run_inference_vllm.ipynb index 40eff1af5155..a4d5e5bf48d2 100644 --- a/examples/notebooks/beam-ml/run_inference_vllm.ipynb +++ b/examples/notebooks/beam-ml/run_inference_vllm.ipynb @@ -338,9 +338,9 @@ "RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 && pip install --upgrade pip\n", "\n", "# Copy the Apache Beam worker dependencies from the Beam Python 3.10 SDK image.\n", - "COPY --from=apache/beam_python3.10_sdk:2.60.0 /opt/apache/beam /opt/apache/beam\n", + "COPY --from=apache/beam_python3.10_sdk:2.61.0 /opt/apache/beam /opt/apache/beam\n", "\n", - "RUN pip install --no-cache-dir -vvv apache-beam[gcp]==2.60.0\n", + "RUN pip install --no-cache-dir -vvv apache-beam[gcp]==2.61.0\n", "RUN pip install openai>=1.52.2 vllm>=0.6.3 triton>=3.1.0\n", "\n", "RUN apt install libcairo2-dev pkg-config python3-dev -y\n",