requestObserver;
protected AbstractWindmillStream(
@@ -132,9 +129,9 @@ private static long debugDuration(long nowMs, long startMs) {
protected abstract boolean hasPendingRequests();
/**
- * Called when the stream is throttled due to resource exhausted errors. Will be called for each
- * resource exhausted error not just the first. onResponse() must stop throttling on receipt of
- * the first good message.
+ * Called when the client side stream is throttled due to resource exhausted errors. Will be
+ * called for each resource exhausted error not just the first. onResponse() must stop throttling
+ * on receipt of the first good message.
*/
protected abstract void startThrottleTimer();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java
index 3737e29efb13..a1f80598d89a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java
@@ -27,23 +27,23 @@
* Used to wrap existing {@link StreamObserver}s to be able to install an {@link
* ClientCallStreamObserver#setOnReadyHandler(Runnable) onReadyHandler}.
*
- *
This is as thread-safe as the undering stream observer that is being wrapped.
+ *
This is as thread-safe as the underlying stream observer that is being wrapped.
*/
-final class ForwardingClientResponseObserver
- implements ClientResponseObserver {
+final class ForwardingClientResponseObserver
+ implements ClientResponseObserver {
private final Runnable onReadyHandler;
private final Runnable onDoneHandler;
- private final StreamObserver inboundObserver;
+ private final StreamObserver inboundObserver;
ForwardingClientResponseObserver(
- StreamObserver inboundObserver, Runnable onReadyHandler, Runnable onDoneHandler) {
+ StreamObserver inboundObserver, Runnable onReadyHandler, Runnable onDoneHandler) {
this.inboundObserver = inboundObserver;
this.onReadyHandler = onReadyHandler;
this.onDoneHandler = onDoneHandler;
}
@Override
- public void onNext(ReqT value) {
+ public void onNext(ResponseT value) {
inboundObserver.onNext(value);
}
@@ -60,7 +60,7 @@ public void onCompleted() {
}
@Override
- public void beforeStart(ClientCallStreamObserver stream) {
+ public void beforeStart(ClientCallStreamObserver stream) {
stream.setOnReadyHandler(onReadyHandler);
}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java
index a046f2fd46ac..e0878b7b0b91 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java
@@ -33,9 +33,9 @@ public static StreamObserverFactory direct(
return new Direct(deadlineSeconds, messagesBetweenIsReadyChecks);
}
- public abstract StreamObserver from(
- Function, StreamObserver> clientFactory,
- StreamObserver responseObserver);
+ public abstract StreamObserver from(
+ Function, StreamObserver> clientFactory,
+ StreamObserver responseObserver);
private static class Direct extends StreamObserverFactory {
private final long deadlineSeconds;
@@ -47,14 +47,14 @@ private static class Direct extends StreamObserverFactory {
}
@Override
- public StreamObserver from(
- Function, StreamObserver> clientFactory,
- StreamObserver inboundObserver) {
+ public StreamObserver from(
+ Function, StreamObserver> clientFactory,
+ StreamObserver inboundObserver) {
AdvancingPhaser phaser = new AdvancingPhaser(1);
- CallStreamObserver outboundObserver =
- (CallStreamObserver)
+ CallStreamObserver outboundObserver =
+ (CallStreamObserver)
clientFactory.apply(
- new ForwardingClientResponseObserver(
+ new ForwardingClientResponseObserver(
inboundObserver, phaser::arrive, phaser::forceTermination));
return new DirectStreamObserver<>(
phaser, outboundObserver, deadlineSeconds, messagesBetweenIsReadyChecks);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillEndpoints.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillEndpoints.java
new file mode 100644
index 000000000000..64b6e675ef5f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillEndpoints.java
@@ -0,0 +1,221 @@
+/*
+ * 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.runners.dataflow.worker.windmill;
+
+import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList.toImmutableList;
+import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap;
+
+import com.google.auto.value.AutoValue;
+import java.net.Inet6Address;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.Optional;
+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.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Value class for holding endpoints used for communicating with Windmill service. Corresponds
+ * directly with {@link Windmill.WorkerMetadataResponse}.
+ */
+@AutoValue
+public abstract class WindmillEndpoints {
+ private static final Logger LOG = LoggerFactory.getLogger(WindmillEndpoints.class);
+
+ /**
+ * Used by GetData GlobalDataRequest(s) to support Beam side inputs. Returns a map where the key
+ * is a global data tag and the value is the endpoint where the data associated with the global
+ * data tag resides.
+ *
+ * @see Beam Side
+ * Inputs
+ */
+ public abstract ImmutableMap globalDataEndpoints();
+
+ /**
+ * Used by GetWork/GetData/CommitWork calls to send, receive, and commit work directly to/from
+ * Windmill servers. Returns a list of endpoints used to communicate with the corresponding
+ * Windmill servers.
+ */
+ public abstract ImmutableList windmillEndpoints();
+
+ public static WindmillEndpoints from(
+ Windmill.WorkerMetadataResponse workerMetadataResponseProto) {
+ ImmutableMap globalDataServers =
+ workerMetadataResponseProto.getGlobalDataEndpointsMap().entrySet().stream()
+ .collect(
+ toImmutableMap(
+ Map.Entry::getKey, // global data key
+ endpoint -> WindmillEndpoints.Endpoint.from(endpoint.getValue())));
+
+ ImmutableList windmillServers =
+ workerMetadataResponseProto.getWorkEndpointsList().stream()
+ .map(WindmillEndpoints.Endpoint::from)
+ .collect(toImmutableList());
+
+ return WindmillEndpoints.builder()
+ .setGlobalDataEndpoints(globalDataServers)
+ .setWindmillEndpoints(windmillServers)
+ .build();
+ }
+
+ public static WindmillEndpoints.Builder builder() {
+ return new AutoValue_WindmillEndpoints.Builder();
+ }
+
+ /**
+ * Representation of an endpoint in {@link Windmill.WorkerMetadataResponse.Endpoint} proto with
+ * the worker_token field, and direct_endpoint field parsed into a {@link WindmillServiceAddress}
+ * which holds either a {@link Inet6Address} or {@link HostAndPort} used to connect to Streaming
+ * Engine. {@link Inet6Address}(s) represent direct Windmill worker connections, and {@link
+ * HostAndPort}(s) represent connections to the Windmill Dispatcher.
+ */
+ @AutoValue
+ public abstract static class Endpoint {
+ /**
+ * {@link WindmillServiceAddress} representation of {@link
+ * Windmill.WorkerMetadataResponse.Endpoint#getDirectEndpoint()}. The proto's direct_endpoint
+ * string can be converted to either {@link Inet6Address} or {@link HostAndPort}.
+ */
+ public abstract Optional directEndpoint();
+
+ /**
+ * Corresponds to {@link Windmill.WorkerMetadataResponse.Endpoint#getWorkerToken()} in the
+ * windmill.proto file.
+ */
+ public abstract Optional workerToken();
+
+ public static Endpoint.Builder builder() {
+ return new AutoValue_WindmillEndpoints_Endpoint.Builder();
+ }
+
+ public static Endpoint from(Windmill.WorkerMetadataResponse.Endpoint endpointProto) {
+ Endpoint.Builder endpointBuilder = Endpoint.builder();
+ if (endpointProto.hasDirectEndpoint() && !endpointProto.getDirectEndpoint().isEmpty()) {
+ parseDirectEndpoint(endpointProto.getDirectEndpoint())
+ .ifPresent(endpointBuilder::setDirectEndpoint);
+ }
+ if (endpointProto.hasWorkerToken() && !endpointProto.getWorkerToken().isEmpty()) {
+ endpointBuilder.setWorkerToken(endpointProto.getWorkerToken());
+ }
+
+ Endpoint endpoint = endpointBuilder.build();
+
+ if (!endpoint.directEndpoint().isPresent() && !endpoint.workerToken().isPresent()) {
+ throw new IllegalArgumentException(
+ String.format(
+ "direct_endpoint=[%s] not present or could not be parsed, and worker_token"
+ + " not present. At least one of these fields is required.",
+ endpointProto.getDirectEndpoint()));
+ }
+
+ return endpoint;
+ }
+
+ @AutoValue.Builder
+ public abstract static class Builder {
+ public abstract Builder setDirectEndpoint(WindmillServiceAddress directEndpoint);
+
+ public abstract Builder setWorkerToken(String workerToken);
+
+ public abstract Endpoint build();
+ }
+ }
+
+ @AutoValue.Builder
+ public abstract static class Builder {
+ public abstract Builder setGlobalDataEndpoints(
+ ImmutableMap globalDataServers);
+
+ public abstract Builder setWindmillEndpoints(
+ ImmutableList windmillServers);
+
+ abstract ImmutableList.Builder windmillEndpointsBuilder();
+
+ public final Builder addWindmillEndpoint(WindmillEndpoints.Endpoint endpoint) {
+ windmillEndpointsBuilder().add(endpoint);
+ return this;
+ }
+
+ public final Builder addAllWindmillEndpoints(Iterable endpoints) {
+ windmillEndpointsBuilder().addAll(endpoints);
+ return this;
+ }
+
+ abstract ImmutableMap.Builder globalDataEndpointsBuilder();
+
+ public final Builder addGlobalDataEndpoint(
+ String globalDataKey, WindmillEndpoints.Endpoint endpoint) {
+ globalDataEndpointsBuilder().put(globalDataKey, endpoint);
+ return this;
+ }
+
+ public final Builder addAllGlobalDataEndpoints(
+ Map globalDataEndpoints) {
+ globalDataEndpointsBuilder().putAll(globalDataEndpoints);
+ return this;
+ }
+
+ public abstract WindmillEndpoints build();
+ }
+
+ private static Optional parseDirectEndpoint(String directEndpoint) {
+ Optional directEndpointIpV6Address =
+ tryParseDirectEndpointIntoIpV6Address(directEndpoint).map(WindmillServiceAddress::create);
+
+ return directEndpointIpV6Address.isPresent()
+ ? directEndpointIpV6Address
+ : tryParseEndpointIntoHostAndPort(directEndpoint).map(WindmillServiceAddress::create);
+ }
+
+ private static Optional tryParseEndpointIntoHostAndPort(String directEndpoint) {
+ try {
+ return Optional.of(HostAndPort.fromString(directEndpoint));
+ } catch (IllegalArgumentException e) {
+ LOG.warn("{} cannot be parsed into a gcpServiceAddress", directEndpoint);
+ return Optional.empty();
+ }
+ }
+
+ private static Optional tryParseDirectEndpointIntoIpV6Address(
+ String directEndpoint) {
+ InetAddress directEndpointAddress = null;
+ try {
+ directEndpointAddress = Inet6Address.getByName(directEndpoint);
+ } catch (UnknownHostException e) {
+ LOG.warn(
+ "Error occurred trying to parse direct_endpoint={} into IPv6 address. Exception={}",
+ directEndpoint,
+ e.toString());
+ }
+
+ // Inet6Address.getByAddress returns either an IPv4 or an IPv6 address depending on the format
+ // of the direct_endpoint string.
+ if (!(directEndpointAddress instanceof Inet6Address)) {
+ LOG.warn(
+ "{} is not an IPv6 address. Direct endpoints are expected to be in IPv6 format.",
+ directEndpoint);
+ return Optional.empty();
+ }
+
+ return Optional.ofNullable((Inet6Address) directEndpointAddress);
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java
new file mode 100644
index 000000000000..3ebda8fab8ed
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServiceAddress.java
@@ -0,0 +1,45 @@
+/*
+ * 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.runners.dataflow.worker.windmill;
+
+import com.google.auto.value.AutoOneOf;
+import java.net.Inet6Address;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
+
+/** Used to create channels to communicate with Streaming Engine via gRpc. */
+@AutoOneOf(WindmillServiceAddress.Kind.class)
+public abstract class WindmillServiceAddress {
+ public static WindmillServiceAddress create(Inet6Address ipv6Address) {
+ return AutoOneOf_WindmillServiceAddress.ipv6(ipv6Address);
+ }
+
+ public static WindmillServiceAddress create(HostAndPort gcpServiceAddress) {
+ return AutoOneOf_WindmillServiceAddress.gcpServiceAddress(gcpServiceAddress);
+ }
+
+ public abstract Kind getKind();
+
+ public abstract Inet6Address ipv6();
+
+ public abstract HostAndPort gcpServiceAddress();
+
+ public enum Kind {
+ IPV6,
+ GCP_SERVICE_ADDRESS
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java
index 70c7cc36ba31..4dd4164fc4ef 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillStream.java
@@ -86,4 +86,8 @@ boolean commitWorkItem(
/** Flushes any pending work items to the wire. */
void flush();
}
+
+ /** Interface for streaming GetWorkerMetadata requests to Windmill. */
+ @ThreadSafe
+ interface GetWorkerMetadataStream extends WindmillStream {}
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java
index 74bd93a5474f..1bba40805dec 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcCommitWorkStream.java
@@ -17,16 +17,17 @@
*/
package org.apache.beam.runners.dataflow.worker.windmill.grpcclient;
+import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+
import java.io.PrintWriter;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
+import java.util.function.Function;
import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream;
-import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc;
import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.CommitStatus;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader;
@@ -37,7 +38,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -56,7 +57,8 @@ final class GrpcCommitWorkStream
private final int streamingRpcBatchLimit;
private GrpcCommitWorkStream(
- CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+ Function, StreamObserver>
+ startCommitWorkRpcFn,
BackOff backoff,
StreamObserverFactory streamObserverFactory,
Set> streamRegistry,
@@ -66,10 +68,7 @@ private GrpcCommitWorkStream(
AtomicLong idGenerator,
int streamingRpcBatchLimit) {
super(
- responseObserver ->
- stub.withDeadlineAfter(
- AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
- .commitWorkStream(responseObserver),
+ startCommitWorkRpcFn,
backoff,
streamObserverFactory,
streamRegistry,
@@ -83,7 +82,8 @@ private GrpcCommitWorkStream(
}
static GrpcCommitWorkStream create(
- CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+ Function, StreamObserver>
+ startCommitWorkRpcFn,
BackOff backoff,
StreamObserverFactory streamObserverFactory,
Set> streamRegistry,
@@ -94,7 +94,7 @@ static GrpcCommitWorkStream create(
int streamingRpcBatchLimit) {
GrpcCommitWorkStream commitWorkStream =
new GrpcCommitWorkStream(
- stub,
+ startCommitWorkRpcFn,
backoff,
streamObserverFactory,
streamRegistry,
@@ -252,7 +252,7 @@ private void issueBatchedRequest(Map requests) {
}
private void issueMultiChunkRequest(final long id, PendingRequest pendingRequest) {
- Preconditions.checkNotNull(pendingRequest.computation);
+ checkNotNull(pendingRequest.computation);
final ByteString serializedCommit = pendingRequest.request.toByteString();
synchronized (this) {
@@ -306,8 +306,13 @@ long getBytes() {
private class Batcher {
- final Map queue = new HashMap<>();
- long queuedBytes = 0;
+ private final Map queue;
+ private long queuedBytes;
+
+ private Batcher() {
+ this.queuedBytes = 0;
+ this.queue = new HashMap<>();
+ }
boolean canAccept(PendingRequest request) {
return queue.isEmpty()
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java
index b51daabb1a2b..238cc771dce8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetDataStream.java
@@ -17,6 +17,9 @@
*/
package org.apache.beam.runners.dataflow.worker.windmill.grpcclient;
+import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Verify.verify;
+
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintWriter;
@@ -28,10 +31,9 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream;
-import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc;
import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationGetDataRequest;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalData;
@@ -45,8 +47,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcGetDataStreamRequests.QueuedBatch;
import org.apache.beam.runners.dataflow.worker.windmill.grpcclient.GrpcGetDataStreamRequests.QueuedRequest;
import org.apache.beam.sdk.util.BackOff;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
-import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Verify;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -64,7 +65,8 @@ final class GrpcGetDataStream
private final int streamingRpcBatchLimit;
private GrpcGetDataStream(
- CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+ Function, StreamObserver>
+ startGetDataRpcFn,
BackOff backoff,
StreamObserverFactory streamObserverFactory,
Set> streamRegistry,
@@ -74,14 +76,7 @@ private GrpcGetDataStream(
AtomicLong idGenerator,
int streamingRpcBatchLimit) {
super(
- responseObserver ->
- stub.withDeadlineAfter(
- AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
- .getDataStream(responseObserver),
- backoff,
- streamObserverFactory,
- streamRegistry,
- logEveryNStreamFailures);
+ startGetDataRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures);
this.idGenerator = idGenerator;
this.getDataThrottleTimer = getDataThrottleTimer;
this.jobHeader = jobHeader;
@@ -91,7 +86,8 @@ private GrpcGetDataStream(
}
static GrpcGetDataStream create(
- CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+ Function, StreamObserver>
+ startGetDataRpcFn,
BackOff backoff,
StreamObserverFactory streamObserverFactory,
Set> streamRegistry,
@@ -102,7 +98,7 @@ static GrpcGetDataStream create(
int streamingRpcBatchLimit) {
GrpcGetDataStream getDataStream =
new GrpcGetDataStream(
- stub,
+ startGetDataRpcFn,
backoff,
streamObserverFactory,
streamRegistry,
@@ -122,7 +118,7 @@ protected synchronized void onNewStream() {
// We rely on close only occurring after all methods on the stream have returned.
// Since the requestKeyedData and requestGlobalData methods are blocking this
// means there should be no pending requests.
- Verify.verify(!hasPendingRequests());
+ verify(!hasPendingRequests());
} else {
for (AppendableInputStream responseStream : pending.values()) {
responseStream.cancel();
@@ -138,14 +134,13 @@ protected boolean hasPendingRequests() {
@Override
@SuppressWarnings("dereference.of.nullable")
protected void onResponse(StreamingGetDataResponse chunk) {
- Preconditions.checkArgument(chunk.getRequestIdCount() == chunk.getSerializedResponseCount());
- Preconditions.checkArgument(
- chunk.getRemainingBytesForResponse() == 0 || chunk.getRequestIdCount() == 1);
+ checkArgument(chunk.getRequestIdCount() == chunk.getSerializedResponseCount());
+ checkArgument(chunk.getRemainingBytesForResponse() == 0 || chunk.getRequestIdCount() == 1);
getDataThrottleTimer.stop();
for (int i = 0; i < chunk.getRequestIdCount(); ++i) {
AppendableInputStream responseStream = pending.get(chunk.getRequestId(i));
- Verify.verify(responseStream != null, "No pending response stream");
+ verify(responseStream != null, "No pending response stream");
responseStream.append(chunk.getSerializedResponse(i).newInput());
if (chunk.getRemainingBytesForResponse() == 0) {
responseStream.complete();
@@ -283,12 +278,12 @@ private void queueRequestAndWait(QueuedRequest request) throws InterruptedExcept
// Finalize the batch so that no additional requests will be added. Leave the batch in the
// queue so that a subsequent batch will wait for it's completion.
synchronized (batches) {
- Verify.verify(batch == batches.peekFirst());
+ verify(batch == batches.peekFirst());
batch.markFinalized();
}
sendBatch(batch.requests());
synchronized (batches) {
- Verify.verify(batch == batches.pollFirst());
+ verify(batch == batches.pollFirst());
}
// Notify all waiters with requests in this batch as well as the sender
// of the next batch (if one exists).
@@ -308,7 +303,7 @@ private void sendBatch(List requests) {
for (QueuedRequest request : requests) {
// Map#put returns null if there was no previous mapping for the key, meaning we have not
// seen it before.
- Verify.verify(pending.put(request.id(), request.getResponseStream()) == null);
+ verify(pending.put(request.id(), request.getResponseStream()) == null);
}
try {
send(batchedRequest);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java
index 6e35beccdb6a..4660fe25b13b 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkStream.java
@@ -23,12 +23,11 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
import javax.annotation.Nullable;
import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils;
import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream;
-import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc;
import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest;
@@ -40,6 +39,7 @@
import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream.WorkItemReceiver;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -58,7 +58,10 @@ final class GrpcGetWorkStream
private final AtomicLong inflightBytes;
private GrpcGetWorkStream(
- CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+ Function<
+ StreamObserver,
+ StreamObserver>
+ startGetWorkRpcFn,
GetWorkRequest request,
BackOff backoff,
StreamObserverFactory streamObserverFactory,
@@ -67,14 +70,7 @@ private GrpcGetWorkStream(
ThrottleTimer getWorkThrottleTimer,
WorkItemReceiver receiver) {
super(
- responseObserver ->
- stub.withDeadlineAfter(
- AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
- .getWorkStream(responseObserver),
- backoff,
- streamObserverFactory,
- streamRegistry,
- logEveryNStreamFailures);
+ startGetWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures);
this.request = request;
this.getWorkThrottleTimer = getWorkThrottleTimer;
this.receiver = receiver;
@@ -84,7 +80,10 @@ private GrpcGetWorkStream(
}
static GrpcGetWorkStream create(
- CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub,
+ Function<
+ StreamObserver,
+ StreamObserver>
+ startGetWorkRpcFn,
GetWorkRequest request,
BackOff backoff,
StreamObserverFactory streamObserverFactory,
@@ -94,7 +93,7 @@ static GrpcGetWorkStream create(
WorkItemReceiver receiver) {
GrpcGetWorkStream getWorkStream =
new GrpcGetWorkStream(
- stub,
+ startGetWorkRpcFn,
request,
backoff,
streamObserverFactory,
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java
new file mode 100644
index 000000000000..427fd412ec7f
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStream.java
@@ -0,0 +1,170 @@
+/*
+ * 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.runners.dataflow.worker.windmill.grpcclient;
+
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import java.io.PrintWriter;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream;
+import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkerMetadataStream;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+final class GrpcGetWorkerMetadataStream
+ extends AbstractWindmillStream
+ implements GetWorkerMetadataStream {
+ private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkerMetadataStream.class);
+ private static final WorkerMetadataRequest HEALTH_CHECK_REQUEST =
+ WorkerMetadataRequest.getDefaultInstance();
+ private final WorkerMetadataRequest workerMetadataRequest;
+ private final ThrottleTimer getWorkerMetadataThrottleTimer;
+ private final Consumer serverMappingConsumer;
+ private final Object metadataLock;
+
+ @GuardedBy("metadataLock")
+ private long metadataVersion;
+
+ @GuardedBy("metadataLock")
+ private WorkerMetadataResponse latestResponse;
+
+ private GrpcGetWorkerMetadataStream(
+ Function, StreamObserver>
+ startGetWorkerMetadataRpcFn,
+ BackOff backoff,
+ StreamObserverFactory streamObserverFactory,
+ Set> streamRegistry,
+ int logEveryNStreamFailures,
+ JobHeader jobHeader,
+ long metadataVersion,
+ ThrottleTimer getWorkerMetadataThrottleTimer,
+ Consumer serverMappingConsumer) {
+ super(
+ startGetWorkerMetadataRpcFn,
+ backoff,
+ streamObserverFactory,
+ streamRegistry,
+ logEveryNStreamFailures);
+ this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build();
+ this.metadataVersion = metadataVersion;
+ this.getWorkerMetadataThrottleTimer = getWorkerMetadataThrottleTimer;
+ this.serverMappingConsumer = serverMappingConsumer;
+ this.latestResponse = WorkerMetadataResponse.getDefaultInstance();
+ this.metadataLock = new Object();
+ }
+
+ public static GrpcGetWorkerMetadataStream create(
+ Function, StreamObserver>
+ startGetWorkerMetadataRpcFn,
+ BackOff backoff,
+ StreamObserverFactory streamObserverFactory,
+ Set> streamRegistry,
+ int logEveryNStreamFailures,
+ JobHeader jobHeader,
+ int metadataVersion,
+ ThrottleTimer getWorkerMetadataThrottleTimer,
+ Consumer serverMappingUpdater) {
+ GrpcGetWorkerMetadataStream getWorkerMetadataStream =
+ new GrpcGetWorkerMetadataStream(
+ startGetWorkerMetadataRpcFn,
+ backoff,
+ streamObserverFactory,
+ streamRegistry,
+ logEveryNStreamFailures,
+ jobHeader,
+ metadataVersion,
+ getWorkerMetadataThrottleTimer,
+ serverMappingUpdater);
+ getWorkerMetadataStream.startStream();
+ return getWorkerMetadataStream;
+ }
+
+ /**
+ * Each instance of {@link AbstractWindmillStream} owns its own responseObserver that calls
+ * onResponse().
+ */
+ @Override
+ protected void onResponse(WorkerMetadataResponse response) {
+ extractWindmillEndpointsFrom(response).ifPresent(serverMappingConsumer);
+ }
+
+ /**
+ * Acquires the {@link #metadataLock} Returns {@link Optional} if the
+ * metadataVersion in the response is not stale (older or equal to {@link #metadataVersion}), else
+ * returns empty {@link Optional}.
+ */
+ private Optional extractWindmillEndpointsFrom(
+ WorkerMetadataResponse response) {
+ synchronized (metadataLock) {
+ if (response.getMetadataVersion() > this.metadataVersion) {
+ this.metadataVersion = response.getMetadataVersion();
+ this.latestResponse = response;
+ return Optional.of(WindmillEndpoints.from(response));
+ } else {
+ // If the currentMetadataVersion is greater than or equal to one in the response, the
+ // response data is stale, and we do not want to do anything.
+ LOG.info(
+ "Received WorkerMetadataResponse={}; Received metadata version={}; Current metadata version={}. "
+ + "Skipping update because received stale metadata",
+ response,
+ response.getMetadataVersion(),
+ this.metadataVersion);
+ }
+ }
+
+ return Optional.empty();
+ }
+
+ @Override
+ protected synchronized void onNewStream() {
+ send(workerMetadataRequest);
+ }
+
+ @Override
+ protected boolean hasPendingRequests() {
+ return false;
+ }
+
+ @Override
+ protected void startThrottleTimer() {
+ getWorkerMetadataThrottleTimer.start();
+ }
+
+ @Override
+ protected void sendHealthCheck() {
+ send(HEALTH_CHECK_REQUEST);
+ }
+
+ @Override
+ protected void appendSpecificHtml(PrintWriter writer) {
+ synchronized (metadataLock) {
+ writer.format(
+ "GetWorkerMetadataStream: version=[%d] , job_header=[%s], latest_response=[%s]",
+ this.metadataVersion, workerMetadataRequest.getHeader(), this.latestResponse);
+ }
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java
index e8745e265eea..19cb90297df5 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcWindmillServer.java
@@ -107,7 +107,6 @@ public final class GrpcWindmillServer extends WindmillServerStub {
private final ThrottleTimer commitWorkThrottleTimer;
private final Random rand;
private final Set> streamRegistry;
-
private ImmutableSet endpoints;
private int logEveryNStreamFailures;
private Duration maxBackoff = MAX_BACKOFF;
@@ -301,14 +300,21 @@ private Channel remoteChannel(HostAndPort endpoint) throws IOException {
.build();
}
+ /**
+ * Stubs returned from this method do not (and should not) have {@link
+ * org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Deadline}(s) set since they represent an absolute
+ * point in time. {@link org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Deadline}(s) should not be
+ * treated as a timeout which represents a relative point in time.
+ *
+ * @see Official gRPC deadline documentation for more
+ * details.
+ */
private synchronized CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub() {
if (stubList.isEmpty()) {
throw new RuntimeException("windmillServiceEndpoint has not been set");
}
- if (stubList.size() == 1) {
- return stubList.get(0);
- }
- return stubList.get(rand.nextInt(stubList.size()));
+
+ return stubList.size() == 1 ? stubList.get(0) : stubList.get(rand.nextInt(stubList.size()));
}
@Override
@@ -398,7 +404,13 @@ public GetWorkStream getWorkStream(GetWorkRequest request, WorkItemReceiver rece
.build();
return GrpcGetWorkStream.create(
- stub(),
+ responseObserver ->
+ stub()
+ // Deadlines are absolute points in time, so generate a new one everytime this
+ // function is called.
+ .withDeadlineAfter(
+ AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
+ .getWorkStream(responseObserver),
getWorkRequest,
grpcBackoff(),
newStreamObserverFactory(),
@@ -411,7 +423,13 @@ public GetWorkStream getWorkStream(GetWorkRequest request, WorkItemReceiver rece
@Override
public GetDataStream getDataStream() {
return GrpcGetDataStream.create(
- stub(),
+ responseObserver ->
+ stub()
+ // Deadlines are absolute points in time, so generate a new one everytime this
+ // function is called.
+ .withDeadlineAfter(
+ AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
+ .getDataStream(responseObserver),
grpcBackoff(),
newStreamObserverFactory(),
streamRegistry,
@@ -425,7 +443,13 @@ public GetDataStream getDataStream() {
@Override
public CommitWorkStream commitWorkStream() {
return GrpcCommitWorkStream.create(
- stub(),
+ responseObserver ->
+ stub()
+ // Deadlines are absolute points in time, so generate a new one everytime this
+ // function is called.
+ .withDeadlineAfter(
+ AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS)
+ .commitWorkStream(responseObserver),
grpcBackoff(),
newStreamObserverFactory(),
streamRegistry,
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
index 8dc7f6217cdc..95b3a43ebf49 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
@@ -33,7 +33,11 @@
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
import com.google.api.services.dataflow.model.CounterUpdate;
import com.google.api.services.dataflow.model.InstructionInput;
@@ -52,7 +56,6 @@
import com.google.api.services.dataflow.model.WriteInstruction;
import java.io.IOException;
import java.io.InputStream;
-import java.time.Clock;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -167,6 +170,7 @@
import org.hamcrest.Matchers;
import org.joda.time.Duration;
import org.joda.time.Instant;
+import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ErrorCollector;
@@ -2851,24 +2855,10 @@ public void testActiveWorkForShardedKeys() throws Exception {
}
@Test
+ @Ignore // Test is flaky on Jenkins (#27555)
public void testMaxThreadMetric() throws Exception {
int maxThreads = 2;
int threadExpiration = 60;
-
- Clock mockClock = Mockito.mock(Clock.class);
- CountDownLatch latch = new CountDownLatch(2);
- doAnswer(
- invocation -> {
- latch.countDown();
- // Return 0 until we are called once (reach max thread count).
- if (latch.getCount() == 1) {
- return 0L;
- }
- return 1000L;
- })
- .when(mockClock)
- .millis();
-
// setting up actual implementation of executor instead of mocking to keep track of
// active thread count.
BoundedQueueExecutor executor =
@@ -2881,8 +2871,7 @@ public void testMaxThreadMetric() throws Exception {
new ThreadFactoryBuilder()
.setNameFormat("DataflowWorkUnits-%d")
.setDaemon(true)
- .build(),
- mockClock);
+ .build());
StreamingDataflowWorker.ComputationState computationState =
new StreamingDataflowWorker.ComputationState(
@@ -2894,17 +2883,15 @@ public void testMaxThreadMetric() throws Exception {
ShardedKey key1Shard1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1);
+ // overriding definition of MockWork to add sleep, which will help us keep track of how
+ // long each work item takes to process and therefore let us manipulate how long the time
+ // at which we're at max threads is.
MockWork m2 =
new MockWork(2) {
@Override
public void run() {
try {
- // Make sure we don't finish before both MockWork are executed, thus afterExecute must
- // be called after
- // beforeExecute.
- while (latch.getCount() > 1) {
- Thread.sleep(50);
- }
+ Thread.sleep(1000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
@@ -2916,9 +2903,7 @@ public void run() {
@Override
public void run() {
try {
- while (latch.getCount() > 1) {
- Thread.sleep(50);
- }
+ Thread.sleep(1000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
@@ -2928,11 +2913,13 @@ public void run() {
assertTrue(computationState.activateWork(key1Shard1, m2));
assertTrue(computationState.activateWork(key1Shard1, m3));
executor.execute(m2, m2.getWorkItem().getSerializedSize());
+
executor.execute(m3, m3.getWorkItem().getSerializedSize());
- // Wait until the afterExecute is called.
- latch.await();
- assertEquals(1000L, executor.allThreadsActiveTime());
+ // Will get close to 1000ms that both work items are processing (sleeping, really)
+ // give or take a few ms.
+ long i = 990L;
+ assertTrue(executor.allThreadsActiveTime() >= i);
executor.shutdown();
}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java
index 4c15da319b12..9f2d5eee8f87 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java
@@ -2169,6 +2169,71 @@ public void testOrderedListMergePendingAdds() {
assertArrayEquals(expected, read);
}
+ @Test
+ public void testOrderedListMergePendingAddsAndDeletes() {
+ SettableFuture, RangeSet>> orderedListFuture = SettableFuture.create();
+ orderedListFuture.set(null);
+ SettableFuture, RangeSet>> deletionsFuture =
+ SettableFuture.create();
+ deletionsFuture.set(null);
+ when(mockReader.valueFuture(
+ systemKey(NAMESPACE, "orderedList" + IdTracker.IDS_AVAILABLE_STR),
+ STATE_FAMILY,
+ IdTracker.IDS_AVAILABLE_CODER))
+ .thenReturn(orderedListFuture);
+ when(mockReader.valueFuture(
+ systemKey(NAMESPACE, "orderedList" + IdTracker.DELETIONS_STR),
+ STATE_FAMILY,
+ IdTracker.SUBRANGE_DELETIONS_CODER))
+ .thenReturn(deletionsFuture);
+
+ SettableFuture>> fromStorage = SettableFuture.create();
+ when(mockReader.orderedListFuture(
+ FULL_ORDERED_LIST_RANGE,
+ key(NAMESPACE, "orderedList"),
+ STATE_FAMILY,
+ StringUtf8Coder.of()))
+ .thenReturn(fromStorage);
+
+ StateTag> addr =
+ StateTags.orderedList("orderedList", StringUtf8Coder.of());
+ OrderedListState orderedListState = underTest.state(NAMESPACE, addr);
+
+ orderedListState.add(TimestampedValue.of("second", Instant.ofEpochMilli(1)));
+ orderedListState.add(TimestampedValue.of("third", Instant.ofEpochMilli(2)));
+ orderedListState.add(TimestampedValue.of("fourth", Instant.ofEpochMilli(2)));
+ orderedListState.add(TimestampedValue.of("eighth", Instant.ofEpochMilli(10)));
+ orderedListState.add(TimestampedValue.of("ninth", Instant.ofEpochMilli(15)));
+
+ orderedListState.clearRange(Instant.ofEpochMilli(2), Instant.ofEpochMilli(5));
+ orderedListState.add(TimestampedValue.of("fourth", Instant.ofEpochMilli(4)));
+
+ fromStorage.set(
+ ImmutableList.of(
+ TimestampedValue.of("first", Instant.ofEpochMilli(-1)),
+ TimestampedValue.of("fifth", Instant.ofEpochMilli(5)),
+ TimestampedValue.of("sixth", Instant.ofEpochMilli(5)),
+ TimestampedValue.of("seventh", Instant.ofEpochMilli(5)),
+ TimestampedValue.of("tenth", Instant.ofEpochMilli(20))));
+
+ TimestampedValue[] expected =
+ Iterables.toArray(
+ ImmutableList.of(
+ TimestampedValue.of("first", Instant.ofEpochMilli(-1)),
+ TimestampedValue.of("second", Instant.ofEpochMilli(1)),
+ TimestampedValue.of("fourth", Instant.ofEpochMilli(4)),
+ TimestampedValue.of("fifth", Instant.ofEpochMilli(5)),
+ TimestampedValue.of("sixth", Instant.ofEpochMilli(5)),
+ TimestampedValue.of("seventh", Instant.ofEpochMilli(5)),
+ TimestampedValue.of("eighth", Instant.ofEpochMilli(10)),
+ TimestampedValue.of("ninth", Instant.ofEpochMilli(15)),
+ TimestampedValue.of("tenth", Instant.ofEpochMilli(20))),
+ TimestampedValue.class);
+
+ TimestampedValue[] read = Iterables.toArray(orderedListState.read(), TimestampedValue.class);
+ assertArrayEquals(expected, read);
+ }
+
@Test
public void testOrderedListPersistEmpty() throws Exception {
StateTag> addr =
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStreamTest.java
new file mode 100644
index 000000000000..45ed3381a8bf
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/grpcclient/GrpcGetWorkerMetadataStreamTest.java
@@ -0,0 +1,328 @@
+/*
+ * 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.runners.dataflow.worker.windmill.grpcclient;
+
+import static com.google.common.truth.Truth.assertThat;
+import static org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream.DEFAULT_STREAM_RPC_DEADLINE_SECONDS;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.dataflow.worker.windmill.AbstractWindmillStream;
+import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc;
+import org.apache.beam.runners.dataflow.worker.windmill.StreamObserverFactory;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataRequest;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkerMetadataResponse;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.testing.GrpcCleanupRule;
+import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.util.MutableHandlerRegistry;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mockito;
+
+@RunWith(JUnit4.class)
+public class GrpcGetWorkerMetadataStreamTest {
+ private static final String IPV6_ADDRESS_1 = "2001:db8:0000:bac5:0000:0000:fed0:81a2";
+ private static final String IPV6_ADDRESS_2 = "2001:db8:0000:bac5:0000:0000:fed0:82a3";
+ private static final List DIRECT_PATH_ENDPOINTS =
+ Lists.newArrayList(
+ WorkerMetadataResponse.Endpoint.newBuilder()
+ .setDirectEndpoint(IPV6_ADDRESS_1)
+ .setWorkerToken("worker_token")
+ .build());
+ private static final Map GLOBAL_DATA_ENDPOINTS =
+ Maps.newHashMap();
+ private static final JobHeader TEST_JOB_HEADER =
+ JobHeader.newBuilder()
+ .setJobId("test_job")
+ .setWorkerId("test_worker")
+ .setProjectId("test_project")
+ .build();
+ private static final String FAKE_SERVER_NAME = "Fake server for GrpcGetWorkerMetadataStreamTest";
+ @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule();
+ private final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry();
+ private final Set> streamRegistry = new HashSet<>();
+ private ManagedChannel inProcessChannel;
+ private GrpcGetWorkerMetadataStream stream;
+
+ private GrpcGetWorkerMetadataStream getWorkerMetadataTestStream(
+ GetWorkerMetadataTestStub getWorkerMetadataTestStub,
+ int metadataVersion,
+ Consumer endpointsConsumer) {
+ serviceRegistry.addService(getWorkerMetadataTestStub);
+ return GrpcGetWorkerMetadataStream.create(
+ responseObserver ->
+ CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel)
+ .getWorkerMetadataStream(responseObserver),
+ FluentBackoff.DEFAULT.backoff(),
+ StreamObserverFactory.direct(DEFAULT_STREAM_RPC_DEADLINE_SECONDS * 2, 1),
+ streamRegistry,
+ 1, // logEveryNStreamFailures
+ TEST_JOB_HEADER,
+ metadataVersion,
+ new ThrottleTimer(),
+ endpointsConsumer);
+ }
+
+ @Before
+ public void setUp() throws IOException {
+ Server server =
+ InProcessServerBuilder.forName(FAKE_SERVER_NAME)
+ .fallbackHandlerRegistry(serviceRegistry)
+ .directExecutor()
+ .build()
+ .start();
+
+ inProcessChannel =
+ grpcCleanup.register(
+ InProcessChannelBuilder.forName(FAKE_SERVER_NAME).directExecutor().build());
+ grpcCleanup.register(server);
+ grpcCleanup.register(inProcessChannel);
+ GLOBAL_DATA_ENDPOINTS.put(
+ "global_data",
+ WorkerMetadataResponse.Endpoint.newBuilder()
+ .setDirectEndpoint(IPV6_ADDRESS_1)
+ .setWorkerToken("worker_token")
+ .build());
+ }
+
+ @After
+ public void cleanUp() {
+ inProcessChannel.shutdownNow();
+ }
+
+ @Test
+ public void testGetWorkerMetadata() {
+ WorkerMetadataResponse mockResponse =
+ WorkerMetadataResponse.newBuilder()
+ .setMetadataVersion(1)
+ .addAllWorkEndpoints(DIRECT_PATH_ENDPOINTS)
+ .putAllGlobalDataEndpoints(GLOBAL_DATA_ENDPOINTS)
+ .build();
+ TestWindmillEndpointsConsumer testWindmillEndpointsConsumer =
+ new TestWindmillEndpointsConsumer();
+ GetWorkerMetadataTestStub testStub =
+ new GetWorkerMetadataTestStub(new TestGetWorkMetadataRequestObserver());
+ int metadataVersion = -1;
+ stream = getWorkerMetadataTestStream(testStub, metadataVersion, testWindmillEndpointsConsumer);
+ testStub.injectWorkerMetadata(mockResponse);
+
+ assertThat(testWindmillEndpointsConsumer.globalDataEndpoints.keySet())
+ .containsExactlyElementsIn(GLOBAL_DATA_ENDPOINTS.keySet());
+ assertThat(testWindmillEndpointsConsumer.windmillEndpoints)
+ .containsExactlyElementsIn(
+ DIRECT_PATH_ENDPOINTS.stream()
+ .map(WindmillEndpoints.Endpoint::from)
+ .collect(Collectors.toList()));
+ }
+
+ @Test
+ public void testGetWorkerMetadata_consumesSubsequentResponseMetadata() {
+ WorkerMetadataResponse initialResponse =
+ WorkerMetadataResponse.newBuilder()
+ .setMetadataVersion(1)
+ .addAllWorkEndpoints(DIRECT_PATH_ENDPOINTS)
+ .putAllGlobalDataEndpoints(GLOBAL_DATA_ENDPOINTS)
+ .build();
+ TestWindmillEndpointsConsumer testWindmillEndpointsConsumer =
+ Mockito.spy(new TestWindmillEndpointsConsumer());
+
+ GetWorkerMetadataTestStub testStub =
+ new GetWorkerMetadataTestStub(new TestGetWorkMetadataRequestObserver());
+ int metadataVersion = 0;
+ stream = getWorkerMetadataTestStream(testStub, metadataVersion, testWindmillEndpointsConsumer);
+ testStub.injectWorkerMetadata(initialResponse);
+
+ List newDirectPathEndpoints =
+ Lists.newArrayList(
+ WorkerMetadataResponse.Endpoint.newBuilder().setDirectEndpoint(IPV6_ADDRESS_2).build());
+ Map newGlobalDataEndpoints = new HashMap<>();
+ newGlobalDataEndpoints.put(
+ "new_global_data",
+ WorkerMetadataResponse.Endpoint.newBuilder().setDirectEndpoint(IPV6_ADDRESS_2).build());
+
+ WorkerMetadataResponse newWorkMetadataResponse =
+ WorkerMetadataResponse.newBuilder()
+ .setMetadataVersion(initialResponse.getMetadataVersion() + 1)
+ .addAllWorkEndpoints(newDirectPathEndpoints)
+ .putAllGlobalDataEndpoints(newGlobalDataEndpoints)
+ .build();
+
+ testStub.injectWorkerMetadata(newWorkMetadataResponse);
+
+ assertThat(newGlobalDataEndpoints.keySet())
+ .containsExactlyElementsIn(testWindmillEndpointsConsumer.globalDataEndpoints.keySet());
+ assertThat(testWindmillEndpointsConsumer.windmillEndpoints)
+ .containsExactlyElementsIn(
+ newDirectPathEndpoints.stream()
+ .map(WindmillEndpoints.Endpoint::from)
+ .collect(Collectors.toList()));
+ }
+
+ @Test
+ public void testGetWorkerMetadata_doesNotConsumeResponseIfMetadataStale() {
+ WorkerMetadataResponse freshEndpoints =
+ WorkerMetadataResponse.newBuilder()
+ .setMetadataVersion(2)
+ .addAllWorkEndpoints(DIRECT_PATH_ENDPOINTS)
+ .putAllGlobalDataEndpoints(GLOBAL_DATA_ENDPOINTS)
+ .build();
+
+ TestWindmillEndpointsConsumer testWindmillEndpointsConsumer =
+ Mockito.spy(new TestWindmillEndpointsConsumer());
+ GetWorkerMetadataTestStub testStub =
+ new GetWorkerMetadataTestStub(new TestGetWorkMetadataRequestObserver());
+ int metadataVersion = 0;
+ stream = getWorkerMetadataTestStream(testStub, metadataVersion, testWindmillEndpointsConsumer);
+ testStub.injectWorkerMetadata(freshEndpoints);
+
+ List staleDirectPathEndpoints =
+ Lists.newArrayList(
+ WorkerMetadataResponse.Endpoint.newBuilder()
+ .setDirectEndpoint("staleWindmillEndpoint")
+ .build());
+ Map staleGlobalDataEndpoints = new HashMap<>();
+ staleGlobalDataEndpoints.put(
+ "stale_global_data",
+ WorkerMetadataResponse.Endpoint.newBuilder().setDirectEndpoint("staleGlobalData").build());
+
+ testStub.injectWorkerMetadata(
+ WorkerMetadataResponse.newBuilder()
+ .setMetadataVersion(1)
+ .addAllWorkEndpoints(staleDirectPathEndpoints)
+ .putAllGlobalDataEndpoints(staleGlobalDataEndpoints)
+ .build());
+
+ // Should have ignored the stale update and only used initial.
+ verify(testWindmillEndpointsConsumer).accept(WindmillEndpoints.from(freshEndpoints));
+ verifyNoMoreInteractions(testWindmillEndpointsConsumer);
+ }
+
+ @Test
+ public void testGetWorkerMetadata_correctlyAddsAndRemovesStreamFromRegistry() {
+ GetWorkerMetadataTestStub testStub =
+ new GetWorkerMetadataTestStub(new TestGetWorkMetadataRequestObserver());
+ stream = getWorkerMetadataTestStream(testStub, 0, new TestWindmillEndpointsConsumer());
+ testStub.injectWorkerMetadata(
+ WorkerMetadataResponse.newBuilder()
+ .setMetadataVersion(1)
+ .addAllWorkEndpoints(DIRECT_PATH_ENDPOINTS)
+ .putAllGlobalDataEndpoints(GLOBAL_DATA_ENDPOINTS)
+ .build());
+
+ assertTrue(streamRegistry.contains(stream));
+ stream.close();
+ assertFalse(streamRegistry.contains(stream));
+ }
+
+ @Test
+ public void testSendHealthCheck() {
+ TestGetWorkMetadataRequestObserver requestObserver =
+ Mockito.spy(new TestGetWorkMetadataRequestObserver());
+ GetWorkerMetadataTestStub testStub = new GetWorkerMetadataTestStub(requestObserver);
+ stream = getWorkerMetadataTestStream(testStub, 0, new TestWindmillEndpointsConsumer());
+ stream.sendHealthCheck();
+
+ verify(requestObserver).onNext(WorkerMetadataRequest.getDefaultInstance());
+ }
+
+ private static class GetWorkerMetadataTestStub
+ extends CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1ImplBase {
+ private final TestGetWorkMetadataRequestObserver requestObserver;
+ private @Nullable StreamObserver responseObserver;
+
+ private GetWorkerMetadataTestStub(TestGetWorkMetadataRequestObserver requestObserver) {
+ this.requestObserver = requestObserver;
+ }
+
+ @Override
+ public StreamObserver getWorkerMetadataStream(
+ StreamObserver responseObserver) {
+ if (this.responseObserver == null) {
+ this.responseObserver = responseObserver;
+ requestObserver.responseObserver = this.responseObserver;
+ }
+
+ return requestObserver;
+ }
+
+ private void injectWorkerMetadata(WorkerMetadataResponse response) {
+ if (responseObserver != null) {
+ responseObserver.onNext(response);
+ }
+ }
+ }
+
+ @SuppressWarnings("UnusedVariable")
+ private static class TestGetWorkMetadataRequestObserver
+ implements StreamObserver {
+ private @Nullable StreamObserver responseObserver;
+
+ @Override
+ public void onNext(WorkerMetadataRequest workerMetadataRequest) {}
+
+ @Override
+ public void onError(Throwable throwable) {}
+
+ @Override
+ public void onCompleted() {
+ responseObserver.onCompleted();
+ }
+ }
+
+ private static class TestWindmillEndpointsConsumer implements Consumer {
+ private final Map globalDataEndpoints;
+ private final Set windmillEndpoints;
+
+ private TestWindmillEndpointsConsumer() {
+ this.globalDataEndpoints = new HashMap<>();
+ this.windmillEndpoints = new HashSet<>();
+ }
+
+ @Override
+ public void accept(WindmillEndpoints windmillEndpoints) {
+ this.globalDataEndpoints.clear();
+ this.windmillEndpoints.clear();
+ this.globalDataEndpoints.putAll(windmillEndpoints.globalDataEndpoints());
+ this.windmillEndpoints.addAll(windmillEndpoints.windmillEndpoints());
+ }
+ }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
index f66b2bed48c6..1759185911d4 100644
--- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
+++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
@@ -746,6 +746,8 @@ message WorkerMetadataRequest {
optional JobHeader header = 1;
}
+// Converted into org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints
+// used to connect to Streaming Engine.
message WorkerMetadataResponse {
// The metadata version increases with every modification. Within a single
// stream it will always be increasing. The version may be used across streams
@@ -758,7 +760,9 @@ message WorkerMetadataResponse {
// CommitWorkStream. Each response on this stream replaces the previous, and
// connections to endpoints that are no longer present should be closed.
message Endpoint {
- optional string endpoint = 1;
+ // IPv6 address of a streaming engine windmill worker.
+ optional string direct_endpoint = 1;
+ optional string worker_token = 2;
}
repeated Endpoint work_endpoints = 2;
@@ -766,10 +770,7 @@ message WorkerMetadataResponse {
// calls to retrieve that global data.
map global_data_endpoints = 3;
- // DirectPath endpoints to be used by user workers for streaming engine jobs.
- // DirectPath endpoints here are virtual IPv6 addresses of the windmill
- // workers.
- repeated Endpoint direct_path_endpoints = 4;
+ reserved 4;
}
service WindmillAppliance {
diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill_service.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill_service.proto
index 803766d1a464..d9183e54e0dd 100644
--- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill_service.proto
+++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill_service.proto
@@ -34,7 +34,7 @@ service CloudWindmillServiceV1Alpha1 {
returns (stream .windmill.StreamingGetWorkResponseChunk);
// Gets worker metadata. Response is a stream.
- rpc GetWorkerMetadataStream(.windmill.WorkerMetadataRequest)
+ rpc GetWorkerMetadataStream(stream .windmill.WorkerMetadataRequest)
returns (stream .windmill.WorkerMetadataResponse);
// Gets data from Windmill.
diff --git a/sdks/go.mod b/sdks/go.mod
index b1e45cee832c..5e91aea021f8 100644
--- a/sdks/go.mod
+++ b/sdks/go.mod
@@ -23,17 +23,17 @@ module github.com/apache/beam/sdks/v2
go 1.20
require (
- cloud.google.com/go/bigquery v1.54.0
+ cloud.google.com/go/bigquery v1.55.0
cloud.google.com/go/bigtable v1.19.0
- cloud.google.com/go/datastore v1.13.0
+ cloud.google.com/go/datastore v1.14.0
cloud.google.com/go/profiler v0.3.1
cloud.google.com/go/pubsub v1.33.0
cloud.google.com/go/spanner v1.49.0
- cloud.google.com/go/storage v1.32.0
+ cloud.google.com/go/storage v1.33.0
github.com/aws/aws-sdk-go-v2 v1.21.0
- github.com/aws/aws-sdk-go-v2/config v1.18.37
- github.com/aws/aws-sdk-go-v2/credentials v1.13.36
- github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.81
+ github.com/aws/aws-sdk-go-v2/config v1.18.39
+ github.com/aws/aws-sdk-go-v2/credentials v1.13.37
+ github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.83
github.com/aws/aws-sdk-go-v2/service/s3 v1.38.5
github.com/aws/smithy-go v1.14.2
github.com/docker/go-connections v0.4.0
@@ -52,14 +52,14 @@ require (
github.com/xitongsys/parquet-go v1.6.2
github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c
go.mongodb.org/mongo-driver v1.12.1
- golang.org/x/net v0.14.0
- golang.org/x/oauth2 v0.11.0
+ golang.org/x/net v0.15.0
+ golang.org/x/oauth2 v0.12.0
golang.org/x/sync v0.3.0
- golang.org/x/sys v0.11.0
- golang.org/x/text v0.12.0
- google.golang.org/api v0.138.0
- google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5
- google.golang.org/grpc v1.57.0
+ golang.org/x/sys v0.12.0
+ golang.org/x/text v0.13.0
+ google.golang.org/api v0.140.0
+ google.golang.org/genproto v0.0.0-20230821184602-ccc8af3d0e93
+ google.golang.org/grpc v1.58.0
google.golang.org/protobuf v1.31.0
gopkg.in/retry.v1 v1.0.3
gopkg.in/yaml.v2 v2.4.0
@@ -74,7 +74,7 @@ require (
require dario.cat/mergo v1.0.0 // indirect
require (
- cloud.google.com/go v0.110.6 // indirect
+ cloud.google.com/go v0.110.7 // indirect
cloud.google.com/go/compute v1.23.0 // indirect
cloud.google.com/go/compute/metadata v0.2.3 // indirect
cloud.google.com/go/iam v1.1.1 // indirect
@@ -97,7 +97,7 @@ require (
github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 // indirect
github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 // indirect
github.com/aws/aws-sdk-go-v2/service/sso v1.13.6 // indirect
- github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.5 // indirect
+ github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.6 // indirect
github.com/aws/aws-sdk-go-v2/service/sts v1.21.5 // indirect
github.com/cenkalti/backoff/v4 v4.2.1 // indirect
github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect
@@ -109,8 +109,8 @@ require (
github.com/docker/distribution v2.8.2+incompatible // indirect
github.com/docker/docker v24.0.5+incompatible // indirect; but required to resolve issue docker has with go1.20
github.com/docker/go-units v0.5.0 // indirect
- github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f // indirect
- github.com/envoyproxy/protoc-gen-validate v0.10.1 // indirect
+ github.com/envoyproxy/go-control-plane v0.11.1 // indirect
+ github.com/envoyproxy/protoc-gen-validate v1.0.2 // indirect
github.com/felixge/httpsnoop v1.0.2 // indirect
github.com/goccy/go-json v0.9.11 // indirect
github.com/gogo/protobuf v1.3.2 // indirect
@@ -119,7 +119,7 @@ require (
github.com/google/flatbuffers v2.0.8+incompatible // indirect
github.com/google/pprof v0.0.0-20221103000818-d260c55eee4c // indirect
github.com/google/renameio/v2 v2.0.0 // indirect
- github.com/google/s2a-go v0.1.5 // indirect
+ github.com/google/s2a-go v0.1.7 // indirect
github.com/googleapis/enterprise-certificate-proxy v0.2.5 // indirect
github.com/googleapis/gax-go/v2 v2.12.0 // indirect
github.com/gorilla/handlers v1.5.1 // indirect
@@ -154,11 +154,11 @@ require (
github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect
github.com/zeebo/xxh3 v1.0.2 // indirect
go.opencensus.io v0.24.0 // indirect
- golang.org/x/crypto v0.12.0 // indirect
+ golang.org/x/crypto v0.13.0 // indirect
golang.org/x/mod v0.11.0 // indirect
- golang.org/x/tools v0.9.1 // indirect
+ golang.org/x/tools v0.10.0 // indirect
golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
google.golang.org/appengine v1.6.7 // indirect
google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5 // indirect
- google.golang.org/genproto/googleapis/rpc v0.0.0-20230807174057-1744710a1577 // indirect
+ google.golang.org/genproto/googleapis/rpc v0.0.0-20230911183012-2d3300fd4832 // indirect
)
diff --git a/sdks/go.sum b/sdks/go.sum
index 1da329ad5f7d..c30891294dbd 100644
--- a/sdks/go.sum
+++ b/sdks/go.sum
@@ -8,13 +8,13 @@ cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg
cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To=
cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4=
cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M=
-cloud.google.com/go v0.110.6 h1:8uYAkj3YHTP/1iwReuHPxLSbdcyc+dSBbzFMrVwDR6Q=
-cloud.google.com/go v0.110.6/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI=
+cloud.google.com/go v0.110.7 h1:rJyC7nWRg2jWGZ4wSJ5nY65GTdYJkg0cd/uXb+ACI6o=
+cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5xsI=
cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o=
cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE=
cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc=
-cloud.google.com/go/bigquery v1.54.0 h1:ify6s7sy+kQuAimRnVTrPUzaeY0+X5GEsKt2C5CiA8w=
-cloud.google.com/go/bigquery v1.54.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec=
+cloud.google.com/go/bigquery v1.55.0 h1:hs44Xxov3XLWQiCx2J8lK5U/ihLqnpm4RVVl5fdtLLI=
+cloud.google.com/go/bigquery v1.55.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec=
cloud.google.com/go/bigtable v1.19.0 h1:wiq9LT0kukfInzvy1joMDijCw/OD1UChpSbORXYn0LI=
cloud.google.com/go/bigtable v1.19.0/go.mod h1:xl5kPa8PTkJjdBxg6qdGH88464nNqmbISHSRU+D2yFE=
cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY=
@@ -24,8 +24,8 @@ cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2Aawl
cloud.google.com/go/datacatalog v1.16.0 h1:qVeQcw1Cz93/cGu2E7TYUPh8Lz5dn5Ws2siIuQ17Vng=
cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk=
-cloud.google.com/go/datastore v1.13.0 h1:ktbC66bOQB3HJPQe8qNI1/aiQ77PMu7hD4mzE6uxe3w=
-cloud.google.com/go/datastore v1.13.0/go.mod h1:KjdB88W897MRITkvWWJrg2OUtrR5XVj1EoLgSp6/N70=
+cloud.google.com/go/datastore v1.14.0 h1:Mq0ApTRdLW3/dyiw+DkjTk0+iGIUvkbzaC8sfPwWTH4=
+cloud.google.com/go/datastore v1.14.0/go.mod h1:GAeStMBIt9bPS7jMJA85kgkpsMkvseWWXiaHya9Jes8=
cloud.google.com/go/iam v1.1.1 h1:lW7fzj15aVIXYHREOqjRBV9PsH0Z6u8Y46a1YGvQP4Y=
cloud.google.com/go/iam v1.1.1/go.mod h1:A5avdyVL2tCppe4unb0951eI9jreack+RJ0/d+KUZOU=
cloud.google.com/go/kms v1.15.0 h1:xYl5WEaSekKYN5gGRyhjvZKM22GVBBCzegGNVPy+aIs=
@@ -43,8 +43,8 @@ cloud.google.com/go/spanner v1.49.0/go.mod h1:eGj9mQGK8+hkgSVbHNQ06pQ4oS+cyc4tXX
cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw=
cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos=
cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk=
-cloud.google.com/go/storage v1.32.0 h1:5w6DxEGOnktmJHarxAOUywxVW9lbNWIzlzzUltG/3+o=
-cloud.google.com/go/storage v1.32.0/go.mod h1:Hhh/dogNRGca7IWv1RC2YqEn0c0G77ctA/OxflYkiD8=
+cloud.google.com/go/storage v1.33.0 h1:PVrDOkIC8qQVa1P3SXGpQvfuJhN2LHOoyZvWs8D2X5M=
+cloud.google.com/go/storage v1.33.0/go.mod h1:Hhh/dogNRGca7IWv1RC2YqEn0c0G77ctA/OxflYkiD8=
dario.cat/mergo v1.0.0 h1:AGCNq9Evsj31mOgNPcLyXc+4PNABt905YmuqPYYpBWk=
dario.cat/mergo v1.0.0/go.mod h1:uNxQE+84aUszobStD9th8a29P2fMDhsBdgRYvZOxGmk=
dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
@@ -67,7 +67,6 @@ github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5
github.com/Microsoft/hcsshim v0.10.0-rc.8 h1:YSZVvlIIDD1UxQpJp0h+dnpLUw+TrY0cx8obKsp3bek=
github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY=
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/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 h1:byKBBF2CKWBjjA4J1ZL2JXttJULvWSl50LegTyRZ728=
github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0=
github.com/apache/arrow/go/v12 v12.0.0 h1:xtZE63VWl7qLdB0JObIXvvhGjoVNrQ9ciIHG2OK5cmc=
@@ -86,18 +85,17 @@ github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pf
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0=
github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM=
github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA=
-github.com/aws/aws-sdk-go-v2/config v1.18.37 h1:RNAfbPqw1CstCooHaTPhScz7z1PyocQj0UL+l95CgzI=
-github.com/aws/aws-sdk-go-v2/config v1.18.37/go.mod h1:8AnEFxW9/XGKCbjYDCJy7iltVNyEI9Iu9qC21UzhhgQ=
+github.com/aws/aws-sdk-go-v2/config v1.18.39 h1:oPVyh6fuu/u4OiW4qcuQyEtk7U7uuNBmHmJSLg1AJsQ=
+github.com/aws/aws-sdk-go-v2/config v1.18.39/go.mod h1:+NH/ZigdPckFpgB1TRcRuWCB/Kbbvkxc/iNAKTq5RhE=
github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc=
-github.com/aws/aws-sdk-go-v2/credentials v1.13.35/go.mod h1:o7rCaLtvK0hUggAGclf76mNGGkaG5a9KWlp+d9IpcV8=
-github.com/aws/aws-sdk-go-v2/credentials v1.13.36 h1:ps0cPswZjpsOk6sLwG6fdXTzrYjCplgPEyG3OUbbdqE=
-github.com/aws/aws-sdk-go-v2/credentials v1.13.36/go.mod h1:sY2phUzxbygoyDtTXhqi7GjGjCQ1S5a5Rj8u3ksBxCg=
+github.com/aws/aws-sdk-go-v2/credentials v1.13.37 h1:BvEdm09+ZEh2XtN+PVHPcYwKY3wIeB6pw7vPRM4M9/U=
+github.com/aws/aws-sdk-go-v2/credentials v1.13.37/go.mod h1:ACLrdkd4CLZyXOghZ8IYumQbcooAcp2jo/s2xsFH8IM=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg=
github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8=
github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4=
-github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.81 h1:PQ9zoe2GEoTVSVPuNtjNrKeVPvyVPWesETMPb7KB3Fk=
-github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.81/go.mod h1:EztVLIU9xGitjdZ1TyHWL9IcNx4952FlqKJe6GLG2z4=
+github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.83 h1:wcluDLIQ0uYaxv0fCWQRimbXkPdTgWHUD21j1CzXEwc=
+github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.83/go.mod h1:nGCBuon134gW67yAtxHKV73x+tAcY/xG4ZPNPDB1h/I=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 h1:22dGT7PneFMx4+b3pz7lMTRyN8ZKH7M2cW4GP9yUS2g=
github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41/go.mod h1:CrObHAuPneJBlfEJ5T3szXOUkLEThaGfvnhTf33buas=
github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 h1:SijA0mgjV8E+8G45ltVHs0fvKpTj8xmZJ3VwhGKtUSI=
@@ -122,11 +120,10 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32
github.com/aws/aws-sdk-go-v2/service/s3 v1.38.5 h1:A42xdtStObqy7NGvzZKpnyNXvoOmm+FENobZ0/ssHWk=
github.com/aws/aws-sdk-go-v2/service/s3 v1.38.5/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM=
github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM=
-github.com/aws/aws-sdk-go-v2/service/sso v1.13.5/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4=
github.com/aws/aws-sdk-go-v2/service/sso v1.13.6 h1:2PylFCfKCEDv6PeSN09pC/VUiRd10wi1VfHG5FrW0/g=
github.com/aws/aws-sdk-go-v2/service/sso v1.13.6/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4=
-github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.5 h1:dnInJb4S0oy8aQuri1mV6ipLlnZPfnsDNB9BGO9PDNY=
-github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.5/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4=
+github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.6 h1:pSB560BbVj9ZlJZF4WYj5zsytWHWKxg+NgyGV4B2L58=
+github.com/aws/aws-sdk-go-v2/service/ssooidc v1.15.6/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4=
github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg=
github.com/aws/aws-sdk-go-v2/service/sts v1.21.5 h1:CQBFElb0LS8RojMJlxRSo/HXipvTZW2S44Lt9Mk2aYQ=
github.com/aws/aws-sdk-go-v2/service/sts v1.21.5/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU=
@@ -138,7 +135,6 @@ github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyY
github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMrBo8f1j86j5WHzznCCQxV/b8g=
github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw=
-github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44=
github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
github.com/checkpoint-restore/go-criu/v5 v5.3.0/go.mod h1:E/eQpaFtUKGOOSEBZgmKAcn+zUUwWxqcaKZlF54wK8E=
@@ -148,13 +144,9 @@ github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMn
github.com/cilium/ebpf v0.7.0/go.mod h1:/oI2+1shJiTGAMgl6/RgJr36Eo1jzrRcAWbcXO2usCA=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc=
-github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk=
-github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI=
github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe h1:QQ3GSy+MqSHxm/d8nCtnAiZdYFd45cYZPs8vOOIYKfk=
github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI=
-github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
-github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 h1:/inchEIKaYC1Akx+H+gqO04wryn5h75LSazbRlnya1k=
github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs=
github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c=
@@ -187,13 +179,11 @@ github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+m
github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4=
github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98=
-github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk=
-github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0=
-github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f h1:7T++XKzy4xg7PKy+bM+Sa9/oe1OC88yz2hXQUISoXfA=
-github.com/envoyproxy/go-control-plane v0.11.1-0.20230524094728-9239064ad72f/go.mod h1:sfYdkwUW4BA3PbKjySwjJy+O4Pu0h62rlqCMHNk+K+Q=
+github.com/envoyproxy/go-control-plane v0.11.1 h1:wSUXTlLfiAQRWs2F+p+EKOY9rUyis1MyGqJ2DIk5HpM=
+github.com/envoyproxy/go-control-plane v0.11.1/go.mod h1:uhMcXKCQMEJHiAb0w+YGefQLaTEw+YhGluxZkrTmD0g=
github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c=
-github.com/envoyproxy/protoc-gen-validate v0.10.1 h1:c0g45+xCJhdgFGw7a5QAfdS4byAbud7miNWJ1WwEVf8=
-github.com/envoyproxy/protoc-gen-validate v0.10.1/go.mod h1:DRjgyB0I43LtJapqN6NiRwroiAU2PaFuvk/vjgh61ss=
+github.com/envoyproxy/protoc-gen-validate v1.0.2 h1:QkIBuU5k+x7/QXPvPPnWXWlCdaBFApVqftFV6k087DA=
+github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE=
github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U=
github.com/felixge/httpsnoop v1.0.2 h1:+nS9g82KMXccJ/wp0zyRW9ZBHFETmMGtkk+2CTTrW4o=
github.com/felixge/httpsnoop v1.0.2/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U=
@@ -203,7 +193,6 @@ github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebP
github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k=
github.com/fsouza/fake-gcs-server v1.47.4 h1:gfBhBxEra20/Om02cvcyL8EnekV8KDb01Yffjat6AKQ=
github.com/fsouza/fake-gcs-server v1.47.4/go.mod h1:vqUZbI12uy9IkRQ54Q4p5AniQsSiUq8alO9Nv2egMmA=
-github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
@@ -241,10 +230,8 @@ github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrU
github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w=
github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0=
github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8=
-github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI=
github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
-github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg=
github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY=
github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
@@ -284,8 +271,8 @@ github.com/google/pprof v0.0.0-20221103000818-d260c55eee4c/go.mod h1:dDKJzRmX4S3
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
github.com/google/renameio/v2 v2.0.0 h1:UifI23ZTGY8Tt29JbYFiuyIU3eX+RNFtUwefq9qAhxg=
github.com/google/renameio/v2 v2.0.0/go.mod h1:BtmJXm5YlszgC+TD4HOEEUFgkJP3nLxehU6hfe7jRt4=
-github.com/google/s2a-go v0.1.5 h1:8IYp3w9nysqv3JH+NJgXJzGbDHzLOTj43BmSkp+O7qg=
-github.com/google/s2a-go v0.1.5/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A=
+github.com/google/s2a-go v0.1.7 h1:60BLSyTrOV4/haCDW4zb1guZItoSq8foHCXrAnjBo/o=
+github.com/google/s2a-go v0.1.7/go.mod h1:50CgR4k1jNlWBu4UfS4AcfhVe1r6pdZPygJ3R8F0Qdw=
github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
github.com/google/uuid v1.3.1 h1:KjJaJ9iWZ3jOFZIf1Lqf4laDRCasjl0BCmnEGxkdLb4=
@@ -300,7 +287,6 @@ github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH
github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q=
github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI=
github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So=
-github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
@@ -394,7 +380,6 @@ github.com/proullon/ramsql v0.1.2 h1:PTtsy2iml/CW3Lsopyr86dlIs7JyYEmfLrfYvQVXD2U
github.com/proullon/ramsql v0.1.2/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E=
github.com/rogpeppe/clock v0.0.0-20190514195947-2896927a307a h1:3QH7VyOaaiUHNrA9Se4YQIRkDTCw1EJls9xTUCaCeRM=
github.com/rogpeppe/clock v0.0.0-20190514195947-2896927a307a/go.mod h1:4r5QyqhjIWCcK8DO4KMclc5Iknq5qVBAlbYYzAbUScQ=
-github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8=
github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc=
@@ -467,7 +452,6 @@ go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0=
go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo=
-go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI=
golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
@@ -477,10 +461,9 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh
golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8=
golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc=
-golang.org/x/crypto v0.0.0-20220314234659-1baeb1ce4c0b/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4=
golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4=
-golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk=
-golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw=
+golang.org/x/crypto v0.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck=
+golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc=
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=
@@ -533,22 +516,21 @@ golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLL
golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
-golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU=
golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg=
golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y=
golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c=
-golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14=
-golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI=
+golang.org/x/net v0.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8=
+golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk=
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=
golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
-golang.org/x/oauth2 v0.11.0 h1:vPL4xzxBM4niKCW6g9whtaWVXTJf1U5e4aZxxFx/gbU=
-golang.org/x/oauth2 v0.11.0/go.mod h1:LdF7O/8bLR/qWK9DrpXmbHLTouvRHK0SgJl0GmDBchk=
+golang.org/x/oauth2 v0.12.0 h1:smVPGxink+n1ZI5pkQa8y6fZT0RW0MgCO5bFpepy4B4=
+golang.org/x/oauth2 v0.12.0/go.mod h1:A74bZ3aGXgCY0qaIC9Ahg6Lglin4AMAco8cIv9baba4=
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=
@@ -580,7 +562,6 @@ golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
-golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200828194041-157a740278f4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -596,8 +577,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
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.11.0 h1:eG7RXZHdqOJ1i+0lgLgCpSXAp6M3LYlAo6osgSi0xOM=
-golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
+golang.org/x/sys v0.12.0 h1:CM0HF96J0hcLAwsHPJZjfdNzs0gftsLfgKt57wWHJ0o=
+golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
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/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
@@ -609,8 +590,8 @@ 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.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ=
golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8=
-golang.org/x/text v0.12.0 h1:k+n5B8goJNdU7hSvEtMUz3d1Q6D/XW4COJSJR6fN0mc=
-golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE=
+golang.org/x/text v0.13.0 h1:ablQoSUd0tRdKxZewP80B+BaqeKJuVhuRxj/dkrun3k=
+golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE=
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=
golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ=
@@ -648,8 +629,8 @@ golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapK
golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA=
golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc=
-golang.org/x/tools v0.9.1 h1:8WMNJAz3zrtPmnYC7ISf5dEn3MT0gY7jBJfw27yrrLo=
-golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc=
+golang.org/x/tools v0.10.0 h1:tvDr/iQoUqNdohiYm0LmmKcBk+q86lb9EprIUFhHHGg=
+golang.org/x/tools v0.10.0/go.mod h1:UJwyiVBsOA2uwvK/e5OY3GTpDUJriEd+/YlqAwLPmyM=
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=
@@ -666,8 +647,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb
google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI=
google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE=
google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE=
-google.golang.org/api v0.138.0 h1:K/tVp05MxNVbHShRw9m7e9VJGdagNeTdMzqPH7AUqr0=
-google.golang.org/api v0.138.0/go.mod h1:4xyob8CxC+0GChNBvEUAk8VBKNvYOTWM9T3v3UfRxuY=
+google.golang.org/api v0.140.0 h1:CaXNdYOH5oQQI7l6iKTHHiMTdxZca4/02hRg2U8c2hM=
+google.golang.org/api v0.140.0/go.mod h1:aGbCiFgtwb2P6badchFbSBUurV6oR5d50Af4iNJtDdI=
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=
@@ -692,14 +673,13 @@ google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvx
google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA=
google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
-google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
-google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5 h1:L6iMMGrtzgHsWofoFcihmDEMYeDR9KN/ThbPWGrh++g=
-google.golang.org/genproto v0.0.0-20230803162519-f966b187b2e5/go.mod h1:oH/ZOT02u4kWEp7oYBGYFFkCdKS/uYR9Z7+0/xuuFp8=
+google.golang.org/genproto v0.0.0-20230821184602-ccc8af3d0e93 h1:zv6ieVm8jNcN33At1+APsRISkRgynuWUxUhv6G123jY=
+google.golang.org/genproto v0.0.0-20230821184602-ccc8af3d0e93/go.mod h1:yZTlhN0tQnXo3h00fuXNCxJdLdIdnVFVBaRJ5LWBbw4=
google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5 h1:nIgk/EEq3/YlnmVVXVnm14rC2oxgs1o0ong4sD/rd44=
google.golang.org/genproto/googleapis/api v0.0.0-20230803162519-f966b187b2e5/go.mod h1:5DZzOUPCLYL3mNkQ0ms0F3EuUNZ7py1Bqeq6sxzI7/Q=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20230807174057-1744710a1577 h1:wukfNtZmZUurLN/atp2hiIeTKn7QJWIQdHzqmsOnAOk=
-google.golang.org/genproto/googleapis/rpc v0.0.0-20230807174057-1744710a1577/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M=
+google.golang.org/genproto/googleapis/rpc v0.0.0-20230911183012-2d3300fd4832 h1:o4LtQxebKIJ4vkzyhtD2rfUNZ20Zf0ik5YVP5E7G7VE=
+google.golang.org/genproto/googleapis/rpc v0.0.0-20230911183012-2d3300fd4832/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M=
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=
@@ -708,12 +688,9 @@ google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQ
google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk=
-google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0=
google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc=
-google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU=
-google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ=
-google.golang.org/grpc v1.57.0 h1:kfzNeI/klCGD2YPMUlaGNT3pxvYfga7smW3Vth8Zsiw=
-google.golang.org/grpc v1.57.0/go.mod h1:Sd+9RMTACXwmub0zcNY2c4arhtrbBYD1AUHI/dt16Mo=
+google.golang.org/grpc v1.58.0 h1:32JY8YpPMSR45K+c3o6b8VL73V+rR8k+DeMIr4vRH8o=
+google.golang.org/grpc v1.58.0/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0=
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=
google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
@@ -743,7 +720,6 @@ gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3M
gopkg.in/retry.v1 v1.0.3 h1:a9CArYczAVv6Qs6VGoLMio99GEs7kY9UzSF9+LD+iGs=
gopkg.in/retry.v1 v1.0.3/go.mod h1:FJkXmWiMaAo7xB+xhvDF59zhfjDWyzmyAxiT4dB688g=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
-gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=
gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY=
gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ=
diff --git a/sdks/go/container/tools/buffered_logging.go b/sdks/go/container/tools/buffered_logging.go
index 5a810dbfdf1a..445d19fabfdc 100644
--- a/sdks/go/container/tools/buffered_logging.go
+++ b/sdks/go/container/tools/buffered_logging.go
@@ -17,24 +17,37 @@ package tools
import (
"context"
+ "log"
+ "math"
"os"
"strings"
+ "time"
)
-const INITIAL_LOG_SIZE int = 255
+const initialLogSize int = 255
// BufferedLogger is a wrapper around the FnAPI logging client meant to be used
// in place of stdout and stderr in bootloader subprocesses. Not intended for
// Beam end users.
type BufferedLogger struct {
- logger *Logger
- builder strings.Builder
- logs []string
+ logger *Logger
+ builder strings.Builder
+ logs []string
+ lastFlush time.Time
+ flushInterval time.Duration
+ periodicFlushContext context.Context
+ now func() time.Time
}
// NewBufferedLogger returns a new BufferedLogger type by reference.
func NewBufferedLogger(logger *Logger) *BufferedLogger {
- return &BufferedLogger{logger: logger}
+ return &BufferedLogger{logger: logger, lastFlush: time.Now(), flushInterval: time.Duration(math.MaxInt64), periodicFlushContext: context.Background(), now: time.Now}
+}
+
+// NewBufferedLoggerWithFlushInterval returns a new BufferedLogger type by reference. This type will
+// flush logs periodically on Write() calls as well as when Flush*() functions are called.
+func NewBufferedLoggerWithFlushInterval(ctx context.Context, logger *Logger, interval time.Duration) *BufferedLogger {
+ return &BufferedLogger{logger: logger, lastFlush: time.Now(), flushInterval: interval, periodicFlushContext: ctx, now: time.Now}
}
// Write implements the io.Writer interface, converting input to a string
@@ -46,10 +59,13 @@ func (b *BufferedLogger) Write(p []byte) (int, error) {
}
n, err := b.builder.Write(p)
if b.logs == nil {
- b.logs = make([]string, 0, INITIAL_LOG_SIZE)
+ b.logs = make([]string, 0, initialLogSize)
}
b.logs = append(b.logs, b.builder.String())
b.builder.Reset()
+ if b.now().Sub(b.lastFlush) > b.flushInterval {
+ b.FlushAtDebug(b.periodicFlushContext)
+ }
return n, err
}
@@ -63,6 +79,7 @@ func (b *BufferedLogger) FlushAtError(ctx context.Context) {
b.logger.Errorf(ctx, message)
}
b.logs = nil
+ b.lastFlush = time.Now()
}
// FlushAtDebug flushes the contents of the buffer to the logging
@@ -75,4 +92,15 @@ func (b *BufferedLogger) FlushAtDebug(ctx context.Context) {
b.logger.Printf(ctx, message)
}
b.logs = nil
+ b.lastFlush = time.Now()
+}
+
+// Prints directly to the logging service. If the logger is nil, prints directly to the
+// console. Used for the container pre-build workflow.
+func (b *BufferedLogger) Printf(ctx context.Context, format string, args ...any) {
+ if b.logger == nil {
+ log.Printf(format, args...)
+ return
+ }
+ b.logger.Printf(ctx, format, args...)
}
diff --git a/sdks/go/container/tools/buffered_logging_test.go b/sdks/go/container/tools/buffered_logging_test.go
index 8feef7b413d3..9f542d2d5ab6 100644
--- a/sdks/go/container/tools/buffered_logging_test.go
+++ b/sdks/go/container/tools/buffered_logging_test.go
@@ -18,6 +18,7 @@ package tools
import (
"context"
"testing"
+ "time"
fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
)
@@ -166,4 +167,75 @@ func TestBufferedLogger(t *testing.T) {
}
}
})
+
+ t.Run("direct print", func(t *testing.T) {
+ catcher := &logCatcher{}
+ l := &Logger{client: catcher}
+ bl := NewBufferedLogger(l)
+
+ bl.Printf(ctx, "foo %v", "bar")
+
+ received := catcher.msgs[0].GetLogEntries()[0]
+
+ if got, want := received.Message, "foo bar"; got != want {
+ t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got message %q, want %q", got, want)
+ }
+
+ if got, want := received.Severity, fnpb.LogEntry_Severity_DEBUG; got != want {
+ t.Errorf("l.Printf(\"foo %%v\", \"bar\"): got severity %v, want %v", got, want)
+ }
+ })
+
+ t.Run("debug flush at interval", func(t *testing.T) {
+ catcher := &logCatcher{}
+ l := &Logger{client: catcher}
+ interval := 5 * time.Second
+ bl := NewBufferedLoggerWithFlushInterval(context.Background(), l, interval)
+
+ startTime := time.Now()
+ bl.now = func() time.Time { return startTime }
+
+ messages := []string{"foo", "bar"}
+
+ for i, message := range messages {
+ if i > 1 {
+ bl.now = func() time.Time { return startTime.Add(6 * time.Second) }
+ }
+ messBytes := []byte(message)
+ n, err := bl.Write(messBytes)
+
+ if err != nil {
+ t.Errorf("got error %v", err)
+ }
+ if got, want := n, len(messBytes); got != want {
+ t.Errorf("got %d bytes written, want %d", got, want)
+ }
+ }
+
+ lastMessage := "baz"
+ bl.now = func() time.Time { return startTime.Add(6 * time.Second) }
+ messBytes := []byte(lastMessage)
+ n, err := bl.Write(messBytes)
+
+ if err != nil {
+ t.Errorf("got error %v", err)
+ }
+ if got, want := n, len(messBytes); got != want {
+ t.Errorf("got %d bytes written, want %d", got, want)
+ }
+
+ // Type should have auto-flushed at debug after the third message
+ received := catcher.msgs[0].GetLogEntries()
+ messages = append(messages, lastMessage)
+
+ for i, message := range received {
+ if got, want := message.Message, messages[i]; got != want {
+ t.Errorf("got message %q, want %q", got, want)
+ }
+
+ if got, want := message.Severity, fnpb.LogEntry_Severity_DEBUG; got != want {
+ t.Errorf("got severity %v, want %v", got, want)
+ }
+ }
+ })
}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/to_string.go b/sdks/go/pkg/beam/core/runtime/exec/to_string.go
new file mode 100644
index 000000000000..2196fd951806
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/to_string.go
@@ -0,0 +1,63 @@
+// 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 exec
+
+import (
+ "context"
+ "fmt"
+)
+
+type ToString struct {
+ // UID is the unit identifier.
+ UID UnitID
+ // Out is the output node.
+ Out Node
+}
+
+func (m *ToString) ID() UnitID {
+ return m.UID
+}
+
+func (m *ToString) Up(ctx context.Context) error {
+ return nil
+}
+
+func (m *ToString) StartBundle(ctx context.Context, id string, data DataContext) error {
+ return m.Out.StartBundle(ctx, id, data)
+}
+
+func (m *ToString) ProcessElement(ctx context.Context, elm *FullValue, values ...ReStream) error {
+ ret := FullValue{
+ Windows: elm.Windows,
+ Elm: elm.Elm,
+ Elm2: fmt.Sprintf("%v", elm.Elm2),
+ Timestamp: elm.Timestamp,
+ }
+
+ return m.Out.ProcessElement(ctx, &ret, values...)
+}
+
+func (m *ToString) FinishBundle(ctx context.Context) error {
+ return m.Out.FinishBundle(ctx)
+}
+
+func (m *ToString) Down(ctx context.Context) error {
+ return nil
+}
+
+func (m *ToString) String() string {
+ return fmt.Sprintf("ToStringFn. Out:%v", m.Out)
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/to_string_test.go b/sdks/go/pkg/beam/core/runtime/exec/to_string_test.go
new file mode 100644
index 000000000000..7f9942a43bd5
--- /dev/null
+++ b/sdks/go/pkg/beam/core/runtime/exec/to_string_test.go
@@ -0,0 +1,57 @@
+// 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 exec
+
+import (
+ "context"
+ "testing"
+)
+
+var toStringTestCases = []struct {
+ Input []any
+ Expected []any
+}{
+ {Input: strInput, Expected: strInput},
+ {Input: intInput, Expected: strInput},
+ {Input: int64Input, Expected: strInput},
+}
+
+func TestToString(t *testing.T) {
+ for _, testCase := range toStringTestCases {
+ ctx := context.Background()
+ out := &CaptureNode{UID: 1}
+ toString := &ToString{UID: 2, Out: out}
+ a := &FixedRoot{UID: 3, Elements: makeKVInput("key", testCase.Input...), Out: toString}
+
+ p, err := NewPlan("a", []Unit{a, toString, out})
+ if err != nil {
+ t.Fatalf("failed to construct plan: %v", err)
+ }
+
+ if err := p.Execute(ctx, "1", DataContext{}); err != nil {
+ t.Fatalf("execute failed: %v", err)
+ }
+
+ if err := p.Down(ctx); err != nil {
+ t.Fatalf("down failed: %v", err)
+ }
+
+ expected := makeKVValues("key", testCase.Expected...)
+ if !equalList(out.Elements, expected) {
+ t.Errorf("tostring returned %v, want %v", extractKeyedValues(out.Elements...), extractKeyedValues(expected...))
+ }
+ }
+}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/translate.go b/sdks/go/pkg/beam/core/runtime/exec/translate.go
index 02a1418880e5..4f078092a310 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/translate.go
@@ -823,6 +823,9 @@ func (b *builder) makeLink(from string, id linkID) (Node, error) {
}
u = sink
+ case graphx.URNToString:
+ u = &ToString{UID: b.idgen.New(), Out: out[0]}
+
default:
panic(fmt.Sprintf("Unexpected transform URN: %v", urn))
}
diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
index ef46fe1e43da..ad76703e3001 100644
--- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
@@ -47,6 +47,7 @@ const (
URNCombinePerKey = "beam:transform:combine_per_key:v1"
URNWindow = "beam:transform:window_into:v1"
URNMapWindows = "beam:transform:map_windows:v1"
+ URNToString = "beam:transform:to_string:v1"
URNIterableSideInput = "beam:side_input:iterable:v1"
URNMultimapSideInput = "beam:side_input:multimap:v1"
@@ -106,6 +107,7 @@ func goCapabilities() []string {
URNWorkerStatus,
URNMonitoringInfoShortID,
URNBaseVersionGo,
+ URNToString,
}
return append(capabilities, knownStandardCoders()...)
}
diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go b/sdks/go/pkg/beam/runners/dataflow/dataflow.go
index 36e418fb5231..7b43ba78f054 100644
--- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go
+++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go
@@ -69,6 +69,7 @@ var (
network = flag.String("network", "", "GCP network (optional)")
subnetwork = flag.String("subnetwork", "", "GCP subnetwork (optional)")
noUsePublicIPs = flag.Bool("no_use_public_ips", false, "Workers must not use public IP addresses (optional)")
+ usePublicIPs = flag.Bool("use_public_ips", true, "Workers must use public IP addresses (optional)")
tempLocation = flag.String("temp_location", "", "Temp location (optional)")
workerMachineType = flag.String("worker_machine_type", "", "GCE machine type (optional)")
machineType = flag.String("machine_type", "", "alias of worker_machine_type (optional)")
@@ -245,6 +246,16 @@ func Execute(ctx context.Context, p *beam.Pipeline) (beam.PipelineResult, error)
return dataflowlib.Execute(ctx, model, opts, workerURL, modelURL, *endpoint, *jobopts.Async)
}
+func isFlagPassed(name string) bool {
+ found := false
+ flag.Visit(func(f *flag.Flag) {
+ if f.Name == name {
+ found = true
+ }
+ })
+ return found
+}
+
func getJobOptions(ctx context.Context, streaming bool) (*dataflowlib.JobOptions, error) {
project := gcpopts.GetProjectFromFlagOrEnvironment(ctx)
if project == "" {
@@ -294,6 +305,17 @@ func getJobOptions(ctx context.Context, streaming bool) (*dataflowlib.JobOptions
return nil, errors.Wrapf(err, "error reading --transform_name_mapping flag as JSON")
}
}
+ if *usePublicIPs == *noUsePublicIPs {
+ useSet := isFlagPassed("use_public_ips")
+ noUseSet := isFlagPassed("no_use_public_ips")
+ // If use_public_ips was explicitly set but no_use_public_ips was not, use that value
+ // We take the explicit value of no_use_public_ips if it was set but use_public_ips was not.
+ if useSet && !noUseSet {
+ *noUsePublicIPs = !*usePublicIPs
+ } else if useSet && noUseSet {
+ return nil, errors.New("exactly one of usePublicIPs and noUsePublicIPs must be true, please check that only one is true")
+ }
+ }
hooks.SerializeHooksToOptions()
diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go b/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go
index d3518964da8a..663695f00c8e 100644
--- a/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go
+++ b/sdks/go/pkg/beam/runners/dataflow/dataflow_test.go
@@ -427,6 +427,81 @@ func TestGetJobOptions_AliasAreEffective(t *testing.T) {
}
}
+func TestGetJobOptions_BadTruePublicIPs(t *testing.T) {
+ resetGlobals()
+ *usePublicIPs = true
+ *noUsePublicIPs = true
+
+ opts, err := getJobOptions(context.Background(), false)
+ if err == nil {
+ t.Error("getJobOptions() returned error nil, want an error")
+ }
+ if opts != nil {
+ t.Errorf("getJobOptions() returned JobOptions when it should not have, got %#v, want nil", opts)
+ }
+}
+
+func TestGetJobOptions_BadFalsePublicIPs(t *testing.T) {
+ resetGlobals()
+ *usePublicIPs = false
+ *noUsePublicIPs = false
+
+ opts, err := getJobOptions(context.Background(), false)
+ if err == nil {
+ t.Error("getJobOptions() returned error nil, want an error")
+ }
+ if opts != nil {
+ t.Errorf("getJobOptions() returned JobOptions when it should not have, got %#v, want nil", opts)
+ }
+}
+
+func TestGetJobOptions_DefaultPublicIPs(t *testing.T) {
+ resetGlobals()
+ *labels = `{"label1": "val1", "label2": "val2"}`
+ *stagingLocation = "gs://testStagingLocation"
+ *minCPUPlatform = "testPlatform"
+ *flexRSGoal = "FLEXRS_SPEED_OPTIMIZED"
+ *dataflowServiceOptions = "opt1,opt2"
+
+ *gcpopts.Project = "testProject"
+ *gcpopts.Region = "testRegion"
+
+ *jobopts.Experiments = "use_runner_v2,use_portable_job_submission"
+ *jobopts.JobName = "testJob"
+
+ opts, err := getJobOptions(context.Background(), false)
+ if err != nil {
+ t.Fatalf("getJobOptions() returned error %q, want %q", err, "nil")
+ }
+ if got, want := opts.NoUsePublicIPs, false; got != want {
+ t.Errorf("getJobOptions().NoUsePublicIPs = %t, want %t", got, want)
+ }
+}
+
+func TestGetJobOptions_NoUsePublicIPs(t *testing.T) {
+ resetGlobals()
+ *labels = `{"label1": "val1", "label2": "val2"}`
+ *stagingLocation = "gs://testStagingLocation"
+ *minCPUPlatform = "testPlatform"
+ *flexRSGoal = "FLEXRS_SPEED_OPTIMIZED"
+ *dataflowServiceOptions = "opt1,opt2"
+ *noUsePublicIPs = true
+
+ *gcpopts.Project = "testProject"
+ *gcpopts.Region = "testRegion"
+
+ *jobopts.Experiments = "use_runner_v2,use_portable_job_submission"
+ *jobopts.JobName = "testJob"
+
+ opts, err := getJobOptions(context.Background(), false)
+ if err != nil {
+ t.Fatalf("getJobOptions() returned error %q, want %q", err, "nil")
+ }
+ if got, want := opts.NoUsePublicIPs, true; got != want {
+ t.Errorf("getJobOptions().NoUsePublicIPs = %t, want %t", got, want)
+ }
+}
+
func getFieldFromOpt(fieldName string, opts *dataflowlib.JobOptions) string {
return reflect.ValueOf(opts).Elem().FieldByName(fieldName).String()
}
@@ -447,6 +522,8 @@ func resetGlobals() {
*stagingLocation = ""
*transformMapping = ""
*update = false
+ *usePublicIPs = true
+ *noUsePublicIPs = false
*workerHarnessImage = ""
*workerMachineType = ""
*machineType = ""
diff --git a/sdks/go/pkg/beam/util/execx/exec.go b/sdks/go/pkg/beam/util/execx/exec.go
index 455b5f5ff84d..aaaf9355e7c1 100644
--- a/sdks/go/pkg/beam/util/execx/exec.go
+++ b/sdks/go/pkg/beam/util/execx/exec.go
@@ -17,6 +17,7 @@
package execx
import (
+ "io"
"os"
"os/exec"
)
@@ -24,16 +25,22 @@ import (
// Execute runs the program with the given arguments. It attaches stdio to the
// child process.
func Execute(prog string, args ...string) error {
- return ExecuteEnv(nil, prog, args...)
+ return ExecuteEnvWithIO(nil, os.Stdin, os.Stdout, os.Stderr, prog, args...)
}
// ExecuteEnv runs the program with the given arguments with additional environment
// variables. It attaches stdio to the child process.
func ExecuteEnv(env map[string]string, prog string, args ...string) error {
+ return ExecuteEnvWithIO(env, os.Stdin, os.Stdout, os.Stderr, prog, args...)
+}
+
+// ExecuteEnvWithIO runs the program with the given arguments with additional environment
+// variables. It attaches custom IO to the child process.
+func ExecuteEnvWithIO(env map[string]string, stdin io.Reader, stdout, stderr io.Writer, prog string, args ...string) error {
cmd := exec.Command(prog, args...)
- cmd.Stdin = os.Stdin
- cmd.Stdout = os.Stdout
- cmd.Stderr = os.Stderr
+ cmd.Stdin = stdin
+ cmd.Stdout = stdout
+ cmd.Stderr = stderr
if env != nil {
cmd.Env = os.Environ()
for k, v := range env {
diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml
index bb8954839d50..7037f0543f4f 100644
--- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml
@@ -20,6 +20,7 @@
+
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java
index ce154a5d5d11..7f2403035d97 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java
@@ -22,6 +22,7 @@
import java.io.Serializable;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Type;
+import java.util.ServiceLoader;
import net.bytebuddy.ByteBuddy;
import net.bytebuddy.asm.AsmVisitorWrapper;
import net.bytebuddy.description.type.TypeDescription;
@@ -35,7 +36,6 @@
import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess;
import net.bytebuddy.jar.asm.ClassWriter;
import net.bytebuddy.matcher.ElementMatchers;
-import org.apache.avro.generic.GenericRecord;
import org.apache.beam.sdk.schemas.JavaFieldSchema.JavaFieldTypeSupplier;
import org.apache.beam.sdk.schemas.NoSuchSchemaException;
import org.apache.beam.sdk.schemas.Schema;
@@ -45,7 +45,6 @@
import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.TypeConversionsFactory;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.util.common.ReflectHelpers;
-import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Primitives;
import org.checkerframework.checker.nullness.qual.Nullable;
@@ -58,6 +57,11 @@
"rawtypes"
})
public class ConvertHelpers {
+ private static class SchemaInformationProviders {
+ private static final ServiceLoader INSTANCE =
+ ServiceLoader.load(SchemaInformationProvider.class);
+ }
+
private static final Logger LOG = LoggerFactory.getLogger(ConvertHelpers.class);
/** Return value after converting a schema. */
@@ -80,17 +84,17 @@ public ConvertedSchemaInformation(
/** Get the coder used for converting from an inputSchema to a given type. */
public static ConvertedSchemaInformation getConvertedSchemaInformation(
Schema inputSchema, TypeDescriptor outputType, SchemaRegistry schemaRegistry) {
- ConvertedSchemaInformation convertedSchema = null;
- if (outputType.equals(TypeDescriptor.of(Row.class))) {
- // If the output is of type Row, then just forward the schema of the input type to the
- // output.
- convertedSchema =
- new ConvertedSchemaInformation<>((SchemaCoder) SchemaCoder.of(inputSchema), null);
- } else if (outputType.equals(TypeDescriptor.of(GenericRecord.class))) {
- convertedSchema =
- new ConvertedSchemaInformation(
- (SchemaCoder) AvroUtils.schemaCoder(AvroUtils.toAvroSchema(inputSchema)), null);
- } else {
+
+ ConvertedSchemaInformation schemaInformation = null;
+ // Try to load schema information from loaded providers
+ for (SchemaInformationProvider provider : SchemaInformationProviders.INSTANCE) {
+ schemaInformation = provider.getConvertedSchemaInformation(inputSchema, outputType);
+ if (schemaInformation != null) {
+ return schemaInformation;
+ }
+ }
+
+ if (schemaInformation == null) {
// Otherwise, try to find a schema for the output type in the schema registry.
Schema outputSchema = null;
SchemaCoder outputSchemaCoder = null;
@@ -129,9 +133,9 @@ public static ConvertedSchemaInformation getConvertedSchemaInformation(
+ outputSchema);
}
}
- convertedSchema = new ConvertedSchemaInformation(outputSchemaCoder, unboxedType);
+ schemaInformation = new ConvertedSchemaInformation(outputSchemaCoder, unboxedType);
}
- return convertedSchema;
+ return schemaInformation;
}
/**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/RowSchemaInformationProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/RowSchemaInformationProvider.java
new file mode 100644
index 000000000000..b9f65c83bafd
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/RowSchemaInformationProvider.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.schemas.utils;
+
+import com.google.auto.service.AutoService;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+@AutoService(SchemaInformationProvider.class)
+public class RowSchemaInformationProvider implements SchemaInformationProvider {
+ @Override
+ @Nullable
+ public ConvertHelpers.ConvertedSchemaInformation getConvertedSchemaInformation(
+ Schema inputSchema, TypeDescriptor outputType) {
+ if (outputType.equals(TypeDescriptor.of(Row.class))) {
+ // If the output is of type Row, then just forward the schema of the input type to the
+ // output.
+ return new ConvertHelpers.ConvertedSchemaInformation<>(
+ (SchemaCoder) SchemaCoder.of(inputSchema), null);
+ }
+ return null;
+ }
+}
diff --git a/.test-infra/jenkins/job_PreCommit_Spotless.groovy b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaInformationProvider.java
similarity index 57%
rename from .test-infra/jenkins/job_PreCommit_Spotless.groovy
rename to sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaInformationProvider.java
index a9da1ad5491a..10727d2a247f 100644
--- a/.test-infra/jenkins/job_PreCommit_Spotless.groovy
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaInformationProvider.java
@@ -15,31 +15,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+package org.apache.beam.sdk.schemas.utils;
-import PrecommitJobBuilder
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.utils.ConvertHelpers.ConvertedSchemaInformation;
+import org.apache.beam.sdk.values.TypeDescriptor;
-PrecommitJobBuilder builder = new PrecommitJobBuilder(
- scope: this,
- nameBase: 'Spotless',
- gradleTask: 'spotlessCheck checkStyleMain checkStyleTest',
- triggerPathPatterns: [
- '^buildSrc/.*$',
- '^sdks/java/.*$',
- '^runners/.*$',
- '^examples/java/.*$',
- '^examples/kotlin/.*$',
- '^.test-infra/jenkins/.*$',
- ]
- )
-builder.build {
- publishers {
- recordIssues {
- tools {
- checkStyle {
- pattern('**/build/reports/checkstyle/*.xml')
- }
- }
- enabledForFailure(true)
- }
- }
+/** Provides an instance of {@link ConvertedSchemaInformation}. Use for internal purposes. */
+@Internal
+public interface SchemaInformationProvider {
+ @Nullable
+ ConvertedSchemaInformation getConvertedSchemaInformation(
+ Schema inputSchema, TypeDescriptor outputType);
}
diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroSchemaInformationProvider.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroSchemaInformationProvider.java
new file mode 100644
index 000000000000..ce426abe69c0
--- /dev/null
+++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroSchemaInformationProvider.java
@@ -0,0 +1,42 @@
+/*
+ * 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.extensions.avro.schemas.utils;
+
+import com.google.auto.service.AutoService;
+import javax.annotation.Nullable;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.utils.ConvertHelpers;
+import org.apache.beam.sdk.schemas.utils.SchemaInformationProvider;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+@AutoService(SchemaInformationProvider.class)
+public class AvroSchemaInformationProvider implements SchemaInformationProvider {
+
+ @Override
+ @Nullable
+ public ConvertHelpers.ConvertedSchemaInformation getConvertedSchemaInformation(
+ Schema inputSchema, TypeDescriptor outputType) {
+ if (outputType.equals(TypeDescriptor.of(GenericRecord.class))) {
+ return new ConvertHelpers.ConvertedSchemaInformation(
+ (SchemaCoder) AvroUtils.schemaCoder(AvroUtils.toAvroSchema(inputSchema)), null);
+ }
+ return null;
+ }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlSeekableTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlSeekableTable.java
index 95f4b7f47f17..7b924cf6b6da 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlSeekableTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlSeekableTable.java
@@ -19,6 +19,8 @@
import java.io.Serializable;
import java.util.List;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.values.Row;
/**
@@ -27,11 +29,17 @@
*/
public interface BeamSqlSeekableTable extends Serializable {
/** prepare the instance. */
- default void setUp() {};
+ default void setUp() {}
+
+ default void startBundle(
+ DoFn.StartBundleContext context, PipelineOptions pipelineOptions) {}
+
+ default void finishBundle(
+ DoFn.FinishBundleContext context, PipelineOptions pipelineOptions) {}
/** return a list of {@code Row} with given key set. */
List seekRow(Row lookupSubRow);
/** cleanup resources of the instance. */
- default void tearDown() {};
+ default void tearDown() {}
}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
index a30822de1519..e4d62c2b5de7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
@@ -26,6 +26,7 @@
import org.apache.beam.sdk.extensions.sql.impl.utils.SerializableRexFieldAccess;
import org.apache.beam.sdk.extensions.sql.impl.utils.SerializableRexInputRef;
import org.apache.beam.sdk.extensions.sql.impl.utils.SerializableRexNode;
+import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.transforms.DoFn;
@@ -155,6 +156,20 @@ public void setup() {
seekableTable.setUp();
}
+ @StartBundle
+ public void startBundle(
+ DoFn.StartBundleContext context,
+ PipelineOptions pipelineOptions) {
+ seekableTable.startBundle(context, pipelineOptions);
+ }
+
+ @FinishBundle
+ public void finishBundle(
+ DoFn.FinishBundleContext context,
+ PipelineOptions pipelineOptions) {
+ seekableTable.finishBundle(context, pipelineOptions);
+ }
+
@ProcessElement
public void processElement(ProcessContext context) {
Row factRow = context.element();
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
index f5f193aecb74..32ee29738bf8 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
@@ -34,7 +34,6 @@
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.NullableCoder;
import org.apache.beam.sdk.coders.ShardedKeyCoder;
-import org.apache.beam.sdk.coders.VoidCoder;
import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
@@ -399,10 +398,12 @@ private WriteResult expandTriggered(PCollection> inpu
"Window Into Global Windows",
Window.>into(new GlobalWindows())
.triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))))
- .apply("Add Void Key", WithKeys.of((Void) null))
- .setCoder(KvCoder.of(VoidCoder.of(), tempTables.getCoder()))
- .apply("GroupByKey", GroupByKey.create())
- .apply("Extract Values", Values.create())
+ // We use this and the following GBK to aggregate by final destination.
+ // This way, each destination has its own pane sequence
+ .apply("AddDestinationKeys", WithKeys.of(result -> result.getKey()))
+ .setCoder(KvCoder.of(destinationCoder, tempTables.getCoder()))
+ .apply("GroupTempTablesByFinalDestination", GroupByKey.create())
+ .apply("ExtractTempTables", Values.create())
.apply(
ParDo.of(
new UpdateSchemaDestination(
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
index 0063952d8b13..00ee815c3c93 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
@@ -689,6 +689,10 @@ public static Row toBeamRow(Schema rowSchema, TableSchema bqSchema, TableRow jso
}
}
+ if (jsonBQValue instanceof byte[] && fieldType.getTypeName() == TypeName.BYTES) {
+ return jsonBQValue;
+ }
+
if (jsonBQValue instanceof List) {
if (fieldType.getCollectionElementType() == null) {
throw new IllegalArgumentException(
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
index 6ee98eb0ddfa..e44617930119 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
@@ -33,6 +33,7 @@
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageApiInsertError;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils;
import org.apache.beam.sdk.io.gcp.bigquery.WriteResult;
import org.apache.beam.sdk.io.gcp.bigquery.providers.BigQueryStorageWriteApiSchemaTransformProvider.BigQueryStorageWriteApiSchemaTransformConfiguration;
@@ -94,7 +95,7 @@ protected SchemaTransform from(
@Override
public String identifier() {
- return String.format("beam:schematransform:org.apache.beam:bigquery_storage_write:v1");
+ return String.format("beam:schematransform:org.apache.beam:bigquery_storage_write:v2");
}
@Override
@@ -125,6 +126,24 @@ public abstract static class BigQueryStorageWriteApiSchemaTransformConfiguration
.put(WriteDisposition.WRITE_APPEND.name(), WriteDisposition.WRITE_APPEND)
.build();
+ @AutoValue
+ public abstract static class ErrorHandling {
+ @SchemaFieldDescription("The name of the output PCollection containing failed writes.")
+ public abstract String getOutput();
+
+ public static Builder builder() {
+ return new AutoValue_BigQueryStorageWriteApiSchemaTransformProvider_BigQueryStorageWriteApiSchemaTransformConfiguration_ErrorHandling
+ .Builder();
+ }
+
+ @AutoValue.Builder
+ public abstract static class Builder {
+ public abstract Builder setOutput(String output);
+
+ public abstract ErrorHandling build();
+ }
+ }
+
public void validate() {
String invalidConfigMessage = "Invalid BigQuery Storage Write configuration: ";
@@ -151,6 +170,12 @@ public void validate() {
this.getWriteDisposition(),
WRITE_DISPOSITIONS.keySet());
}
+
+ if (this.getErrorHandling() != null) {
+ checkArgument(
+ !Strings.isNullOrEmpty(this.getErrorHandling().getOutput()),
+ invalidConfigMessage + "Output must not be empty if error handling specified.");
+ }
}
/**
@@ -198,6 +223,10 @@ public static Builder builder() {
@Nullable
public abstract Boolean getAutoSharding();
+ @SchemaFieldDescription("This option specifies whether and where to output unwritable rows.")
+ @Nullable
+ public abstract ErrorHandling getErrorHandling();
+
/** Builder for {@link BigQueryStorageWriteApiSchemaTransformConfiguration}. */
@AutoValue.Builder
public abstract static class Builder {
@@ -214,6 +243,8 @@ public abstract static class Builder {
public abstract Builder setAutoSharding(Boolean autoSharding);
+ public abstract Builder setErrorHandling(ErrorHandling errorHandling);
+
/** Builds a {@link BigQueryStorageWriteApiSchemaTransformConfiguration} instance. */
public abstract BigQueryStorageWriteApiSchemaTransformProvider
.BigQueryStorageWriteApiSchemaTransformConfiguration
@@ -244,7 +275,7 @@ public void setBigQueryServices(BigQueryServices testBigQueryServices) {
// A generic counter for PCollection of Row. Will be initialized with the given
// name argument. Performs element-wise counter of the input PCollection.
- private static class ElementCounterFn extends DoFn {
+ private static class ElementCounterFn extends DoFn {
private Counter bqGenericElementCounter;
private Long elementsInBundle = 0L;
@@ -267,6 +298,18 @@ public void finish(FinishBundleContext c) {
}
}
+ private static class FailOnError extends DoFn {
+ @ProcessElement
+ public void process(ProcessContext c) {
+ throw new RuntimeException(c.element().getErrorMessage());
+ }
+ }
+
+ private static class NoOutputDoFn extends DoFn {
+ @ProcessElement
+ public void process(ProcessContext c) {}
+ }
+
@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
// Check that the input exists
@@ -294,53 +337,55 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) {
WriteResult result =
inputRows
.apply(
- "element-count", ParDo.of(new ElementCounterFn("BigQuery-write-element-counter")))
+ "element-count",
+ ParDo.of(new ElementCounterFn("BigQuery-write-element-counter")))
.setRowSchema(inputSchema)
.apply(write);
- Schema rowSchema = inputRows.getSchema();
- Schema errorSchema =
- Schema.of(
- Field.of("failed_row", FieldType.row(rowSchema)),
- Field.of("error_message", FieldType.STRING));
-
- // Failed rows
- PCollection failedRows =
- result
- .getFailedStorageApiInserts()
- .apply(
- "Construct failed rows",
- MapElements.into(TypeDescriptors.rows())
- .via(
- (storageError) ->
- BigQueryUtils.toBeamRow(rowSchema, storageError.getRow())))
- .setRowSchema(rowSchema);
-
- // Failed rows with error message
- PCollection failedRowsWithErrors =
+ // Give something that can be followed.
+ PCollection postWrite =
result
.getFailedStorageApiInserts()
- .apply(
- "Construct failed rows and errors",
- MapElements.into(TypeDescriptors.rows())
- .via(
- (storageError) ->
- Row.withSchema(errorSchema)
- .withFieldValue("error_message", storageError.getErrorMessage())
- .withFieldValue(
- "failed_row",
- BigQueryUtils.toBeamRow(rowSchema, storageError.getRow()))
- .build()))
- .setRowSchema(errorSchema);
-
- PCollection failedRowsOutput =
- failedRows
- .apply("error-count", ParDo.of(new ElementCounterFn("BigQuery-write-error-counter")))
- .setRowSchema(rowSchema);
-
- return PCollectionRowTuple.of(FAILED_ROWS_TAG, failedRowsOutput)
- .and(FAILED_ROWS_WITH_ERRORS_TAG, failedRowsWithErrors)
- .and("errors", failedRowsWithErrors);
+ .apply("post-write", ParDo.of(new NoOutputDoFn()))
+ .setRowSchema(Schema.of());
+
+ if (configuration.getErrorHandling() == null) {
+ result
+ .getFailedStorageApiInserts()
+ .apply("Error on failed inserts", ParDo.of(new FailOnError()));
+ return PCollectionRowTuple.of("post_write", postWrite);
+ } else {
+ result
+ .getFailedStorageApiInserts()
+ .apply(
+ "error-count",
+ ParDo.of(
+ new ElementCounterFn(
+ "BigQuery-write-error-counter")));
+
+ // Failed rows with error message
+ Schema errorSchema =
+ Schema.of(
+ Field.of("failed_row", FieldType.row(inputSchema)),
+ Field.of("error_message", FieldType.STRING));
+ PCollection failedRowsWithErrors =
+ result
+ .getFailedStorageApiInserts()
+ .apply(
+ "Construct failed rows and errors",
+ MapElements.into(TypeDescriptors.rows())
+ .via(
+ (storageError) ->
+ Row.withSchema(errorSchema)
+ .withFieldValue("error_message", storageError.getErrorMessage())
+ .withFieldValue(
+ "failed_row",
+ BigQueryUtils.toBeamRow(inputSchema, storageError.getRow()))
+ .build()))
+ .setRowSchema(errorSchema);
+ return PCollectionRowTuple.of("post_write", postWrite)
+ .and(configuration.getErrorHandling().getOutput(), failedRowsWithErrors);
+ }
}
BigQueryIO.Write createStorageWriteApiTransform() {
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
index 9bff77a16588..f4074cc1a556 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
@@ -87,6 +87,7 @@ public class BigQueryUtilsTest {
.addNullableField("time0s_0ns", Schema.FieldType.logicalType(SqlTypes.TIME))
.addNullableField("valid", Schema.FieldType.BOOLEAN)
.addNullableField("binary", Schema.FieldType.BYTES)
+ .addNullableField("raw_bytes", Schema.FieldType.BYTES)
.addNullableField("numeric", Schema.FieldType.DECIMAL)
.addNullableField("boolean", Schema.FieldType.BOOLEAN)
.addNullableField("long", Schema.FieldType.INT64)
@@ -188,6 +189,9 @@ public class BigQueryUtilsTest {
private static final TableFieldSchema BINARY =
new TableFieldSchema().setName("binary").setType(StandardSQLTypeName.BYTES.toString());
+ private static final TableFieldSchema RAW_BYTES =
+ new TableFieldSchema().setName("raw_bytes").setType(StandardSQLTypeName.BYTES.toString());
+
private static final TableFieldSchema NUMERIC =
new TableFieldSchema().setName("numeric").setType(StandardSQLTypeName.NUMERIC.toString());
@@ -246,6 +250,7 @@ public class BigQueryUtilsTest {
TIME_0S_0NS,
VALID,
BINARY,
+ RAW_BYTES,
NUMERIC,
BOOLEAN,
LONG,
@@ -276,6 +281,7 @@ public class BigQueryUtilsTest {
TIME_0S_0NS,
VALID,
BINARY,
+ RAW_BYTES,
NUMERIC,
BOOLEAN,
LONG,
@@ -316,6 +322,7 @@ public class BigQueryUtilsTest {
LocalTime.parse("12:34"),
false,
Base64.getDecoder().decode("ABCD1234"),
+ Base64.getDecoder().decode("ABCD1234"),
new BigDecimal("123.456").setScale(3, RoundingMode.HALF_UP),
true,
123L,
@@ -346,6 +353,7 @@ public class BigQueryUtilsTest {
.set("time0s_0ns", "12:34:00")
.set("valid", "false")
.set("binary", "ABCD1234")
+ .set("raw_bytes", Base64.getDecoder().decode("ABCD1234"))
.set("numeric", "123.456")
.set("boolean", true)
.set("long", 123L)
@@ -355,7 +363,7 @@ public class BigQueryUtilsTest {
Row.withSchema(FLAT_TYPE)
.addValues(
null, null, null, null, null, null, null, null, null, null, null, null, null, null,
- null, null, null, null, null, null, null, null)
+ null, null, null, null, null, null, null, null, null)
.build();
private static final TableRow BQ_NULL_FLAT_ROW =
@@ -378,6 +386,7 @@ public class BigQueryUtilsTest {
.set("time0s_0ns", null)
.set("valid", null)
.set("binary", null)
+ .set("raw_bytes", null)
.set("numeric", null)
.set("boolean", null)
.set("long", null)
@@ -457,6 +466,7 @@ public class BigQueryUtilsTest {
TIME_0S_0NS,
VALID,
BINARY,
+ RAW_BYTES,
NUMERIC,
BOOLEAN,
LONG,
@@ -512,6 +522,7 @@ public void testToTableSchema_flat() {
TIME_0S_0NS,
VALID,
BINARY,
+ RAW_BYTES,
NUMERIC,
BOOLEAN,
LONG,
@@ -562,6 +573,7 @@ public void testToTableSchema_row() {
TIME_0S_0NS,
VALID,
BINARY,
+ RAW_BYTES,
NUMERIC,
BOOLEAN,
LONG,
@@ -598,6 +610,7 @@ public void testToTableSchema_array_row() {
TIME_0S_0NS,
VALID,
BINARY,
+ RAW_BYTES,
NUMERIC,
BOOLEAN,
LONG,
@@ -620,7 +633,7 @@ public void testToTableSchema_map() {
public void testToTableRow_flat() {
TableRow row = toTableRow().apply(FLAT_ROW);
- assertThat(row.size(), equalTo(22));
+ assertThat(row.size(), equalTo(23));
assertThat(row, hasEntry("id", "123"));
assertThat(row, hasEntry("value", "123.456"));
assertThat(row, hasEntry("datetime", "2020-11-02T12:34:56.789876"));
@@ -635,6 +648,7 @@ public void testToTableRow_flat() {
assertThat(row, hasEntry("name", "test"));
assertThat(row, hasEntry("valid", "false"));
assertThat(row, hasEntry("binary", "ABCD1234"));
+ assertThat(row, hasEntry("raw_bytes", "ABCD1234"));
assertThat(row, hasEntry("numeric", "123.456"));
assertThat(row, hasEntry("boolean", "true"));
assertThat(row, hasEntry("long", "123"));
@@ -674,7 +688,7 @@ public void testToTableRow_row() {
assertThat(row.size(), equalTo(1));
row = (TableRow) row.get("row");
- assertThat(row.size(), equalTo(22));
+ assertThat(row.size(), equalTo(23));
assertThat(row, hasEntry("id", "123"));
assertThat(row, hasEntry("value", "123.456"));
assertThat(row, hasEntry("datetime", "2020-11-02T12:34:56.789876"));
@@ -689,6 +703,7 @@ public void testToTableRow_row() {
assertThat(row, hasEntry("name", "test"));
assertThat(row, hasEntry("valid", "false"));
assertThat(row, hasEntry("binary", "ABCD1234"));
+ assertThat(row, hasEntry("raw_bytes", "ABCD1234"));
assertThat(row, hasEntry("numeric", "123.456"));
assertThat(row, hasEntry("boolean", "true"));
assertThat(row, hasEntry("long", "123"));
@@ -701,7 +716,7 @@ public void testToTableRow_array_row() {
assertThat(row.size(), equalTo(1));
row = ((List) row.get("rows")).get(0);
- assertThat(row.size(), equalTo(22));
+ assertThat(row.size(), equalTo(23));
assertThat(row, hasEntry("id", "123"));
assertThat(row, hasEntry("value", "123.456"));
assertThat(row, hasEntry("datetime", "2020-11-02T12:34:56.789876"));
@@ -716,6 +731,7 @@ public void testToTableRow_array_row() {
assertThat(row, hasEntry("name", "test"));
assertThat(row, hasEntry("valid", "false"));
assertThat(row, hasEntry("binary", "ABCD1234"));
+ assertThat(row, hasEntry("raw_bytes", "ABCD1234"));
assertThat(row, hasEntry("numeric", "123.456"));
assertThat(row, hasEntry("boolean", "true"));
assertThat(row, hasEntry("long", "123"));
@@ -726,7 +742,7 @@ public void testToTableRow_array_row() {
public void testToTableRow_null_row() {
TableRow row = toTableRow().apply(NULL_FLAT_ROW);
- assertThat(row.size(), equalTo(22));
+ assertThat(row.size(), equalTo(23));
assertThat(row, hasEntry("id", null));
assertThat(row, hasEntry("value", null));
assertThat(row, hasEntry("name", null));
@@ -745,6 +761,7 @@ public void testToTableRow_null_row() {
assertThat(row, hasEntry("time0s_0ns", null));
assertThat(row, hasEntry("valid", null));
assertThat(row, hasEntry("binary", null));
+ assertThat(row, hasEntry("raw_bytes", null));
assertThat(row, hasEntry("numeric", null));
assertThat(row, hasEntry("boolean", null));
assertThat(row, hasEntry("long", null));
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java
new file mode 100644
index 000000000000..012afed6fb43
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java
@@ -0,0 +1,497 @@
+/*
+ * 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.gcp.bigquery;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assume.assumeTrue;
+
+import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableFieldSchema;
+import com.google.api.services.bigquery.model.TableReference;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.api.services.bigquery.model.TableSchema;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
+import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.io.gcp.testing.BigqueryClient;
+import org.apache.beam.sdk.options.ExperimentalOptions;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PeriodicImpulse;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.sdk.values.ValueInSingleWindow;
+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;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(Parameterized.class)
+public class FileLoadsStreamingIT {
+ private static final Logger LOG = LoggerFactory.getLogger(FileLoadsStreamingIT.class);
+
+ @Parameterized.Parameters
+ public static Iterable data() {
+ return ImmutableList.of(new Object[] {false}, new Object[] {true});
+ }
+
+ @Parameterized.Parameter(0)
+ public boolean useInputSchema;
+
+ @Rule public TestName testName = new TestName();
+
+ private static final BigqueryClient BQ_CLIENT = new BigqueryClient("FileLoadsStreamingIT");
+ private static final String PROJECT =
+ TestPipeline.testingPipelineOptions().as(GcpOptions.class).getProject();
+ private static final String BIG_QUERY_DATASET_ID = "file_loads_streaming_it_" + System.nanoTime();
+
+ private static final String[] FIELDS = {
+ "BOOL",
+ "BOOLEAN",
+ "BYTES",
+ "INT64",
+ "INTEGER",
+ "FLOAT",
+ "FLOAT64",
+ "NUMERIC",
+ "STRING",
+ "DATE",
+ "TIMESTAMP"
+ };
+
+ private static final int TOTAL_N = 50;
+
+ private final Random randomGenerator = new Random();
+
+ @BeforeClass
+ public static void setUpTestEnvironment() throws IOException, InterruptedException {
+ // Create one BQ dataset for all test cases.
+ cleanUp();
+ BQ_CLIENT.createNewDataset(PROJECT, BIG_QUERY_DATASET_ID);
+ }
+
+ @AfterClass
+ public static void cleanUp() {
+ BQ_CLIENT.deleteDataset(PROJECT, BIG_QUERY_DATASET_ID);
+ }
+
+ static class GenerateRowFunc implements SerializableFunction {
+ private final List fieldNames;
+
+ public GenerateRowFunc(List fieldNames) {
+ this.fieldNames = fieldNames;
+ }
+
+ @Override
+ public TableRow apply(Long rowId) {
+ TableRow row = new TableRow();
+ row.set("id", rowId);
+
+ for (String name : fieldNames) {
+ String type = Iterables.get(Splitter.on('_').split(name), 0);
+ switch (type) {
+ case "BOOL":
+ case "BOOLEAN":
+ if (rowId % 2 == 0) {
+ row.set(name, false);
+ } else {
+ row.set(name, true);
+ }
+ break;
+ case "BYTES":
+ row.set(name, String.format("test_blob_%s", rowId).getBytes(StandardCharsets.UTF_8));
+ break;
+ case "INT64":
+ case "INTEGER":
+ row.set(name, String.valueOf(rowId + 10));
+ break;
+ case "FLOAT":
+ case "FLOAT64":
+ row.set(name, String.valueOf(0.5 + rowId));
+ break;
+ case "NUMERIC":
+ row.set(name, String.valueOf(rowId + 0.12345));
+ break;
+ case "DATE":
+ row.set(name, "2022-01-01");
+ break;
+ case "TIMESTAMP":
+ row.set(name, "2022-01-01 10:10:10.012 UTC");
+ break;
+ case "STRING":
+ row.set(name, "test_string" + rowId);
+ break;
+ default:
+ row.set(name, "unknown" + rowId);
+ break;
+ }
+ }
+ return row;
+ }
+ }
+
+ private static TableSchema makeTableSchemaFromTypes(List fieldNames) {
+ ImmutableList.Builder builder = ImmutableList.builder();
+
+ // Add an id field for verification of correctness
+ builder.add(new TableFieldSchema().setType("INTEGER").setName("id").setMode("REQUIRED"));
+
+ // the name is prefix with type_.
+ for (String name : fieldNames) {
+ String mode = "REQUIRED";
+ builder.add(new TableFieldSchema().setType(name).setName(name).setMode(mode));
+ }
+
+ return new TableSchema().setFields(builder.build());
+ }
+
+ private String maybeCreateTable(TableSchema tableSchema, String suffix)
+ throws IOException, InterruptedException {
+ String tableId = Iterables.get(Splitter.on('[').split(testName.getMethodName()), 0);
+
+ BQ_CLIENT.deleteTable(PROJECT, BIG_QUERY_DATASET_ID, tableId + suffix);
+ if (!useInputSchema) {
+ BQ_CLIENT.createNewTable(
+ PROJECT,
+ BIG_QUERY_DATASET_ID,
+ new Table()
+ .setSchema(tableSchema)
+ .setTableReference(
+ new TableReference()
+ .setTableId(tableId + suffix)
+ .setDatasetId(BIG_QUERY_DATASET_ID)
+ .setProjectId(PROJECT)));
+ } else {
+ tableId += "WithInputSchema";
+ }
+ return String.format("%s.%s.%s", PROJECT, BIG_QUERY_DATASET_ID, tableId + suffix);
+ }
+
+ private void runStreaming(int numFileShards, boolean useCopyJobs)
+ throws IOException, InterruptedException {
+ TestPipelineOptions opts = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class);
+ opts.setTempLocation(opts.getTempRoot());
+ Pipeline p = Pipeline.create(opts);
+
+ // Only run the most relevant test case on Dataflow.
+ // Testing this dimension on DirectRunner is sufficient
+ if (p.getOptions().getRunner().getName().contains("DataflowRunner")) {
+ assumeTrue("Skipping in favor of more relevant test case", useInputSchema);
+ // Need to manually enable streaming engine for legacy dataflow runner
+ ExperimentalOptions.addExperiment(
+ p.getOptions().as(ExperimentalOptions.class), GcpOptions.STREAMING_ENGINE_EXPERIMENT);
+ }
+
+ List fieldNamesOrigin = Arrays.asList(FIELDS);
+ // Shuffle the fields in the write schema to do fuzz testing on field order
+ List fieldNamesShuffled = new ArrayList(fieldNamesOrigin);
+ Collections.shuffle(fieldNamesShuffled, randomGenerator);
+
+ TableSchema bqTableSchema = makeTableSchemaFromTypes(fieldNamesOrigin);
+ TableSchema inputSchema = makeTableSchemaFromTypes(fieldNamesShuffled);
+ String tableSpec = maybeCreateTable(bqTableSchema, "");
+
+ // set up and build pipeline
+ Instant start = new Instant(0);
+ GenerateRowFunc generateRowFunc = new GenerateRowFunc(fieldNamesShuffled);
+ PCollection instants =
+ p.apply(
+ "Generate Instants",
+ PeriodicImpulse.create()
+ .startAt(start)
+ .stopAt(start.plus(Duration.standardSeconds(TOTAL_N - 1)))
+ .withInterval(Duration.standardSeconds(1))
+ .catchUpToNow(false));
+ PCollection rows =
+ instants.apply(
+ "Create TableRows",
+ MapElements.into(TypeDescriptor.of(TableRow.class))
+ .via(instant -> generateRowFunc.apply(instant.getMillis() / 1000)));
+ // build write transform
+ Write write =
+ BigQueryIO.writeTableRows()
+ .to(tableSpec)
+ .withMethod(Write.Method.FILE_LOADS)
+ .withTriggeringFrequency(Duration.standardSeconds(10));
+ if (useCopyJobs) {
+ write = write.withMaxBytesPerPartition(250);
+ }
+ if (useInputSchema) {
+ // we're creating the table with the input schema
+ write =
+ write
+ .withSchema(inputSchema)
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+ .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE);
+ } else {
+ // table already exists with a schema, no need to create it
+ write =
+ write
+ .withCreateDisposition(CreateDisposition.CREATE_NEVER)
+ .withWriteDisposition(WriteDisposition.WRITE_APPEND);
+ }
+ write = numFileShards == 0 ? write.withAutoSharding() : write.withNumFileShards(numFileShards);
+
+ rows.apply("Stream loads to BigQuery", write);
+ p.run().waitUntilFinish();
+
+ List expectedRows = new ArrayList<>();
+ for (long i = 0; i < TOTAL_N; i++) {
+ expectedRows.add(generateRowFunc.apply(i));
+ }
+
+ // Perform checks
+ checkRowCompleteness(tableSpec, inputSchema, expectedRows);
+ }
+
+ // Check that the expected rows reached the table.
+ private static void checkRowCompleteness(
+ String tableSpec, TableSchema schema, List expectedRows)
+ throws IOException, InterruptedException {
+ List actualTableRows =
+ BQ_CLIENT.queryUnflattened(
+ String.format("SELECT * FROM [%s]", tableSpec), PROJECT, true, false);
+
+ Schema rowSchema = BigQueryUtils.fromTableSchema(schema);
+ List actualBeamRows =
+ actualTableRows.stream()
+ .map(tableRow -> BigQueryUtils.toBeamRow(rowSchema, tableRow))
+ .collect(Collectors.toList());
+ List expectedBeamRows =
+ expectedRows.stream()
+ .map(tableRow -> BigQueryUtils.toBeamRow(rowSchema, tableRow))
+ .collect(Collectors.toList());
+ LOG.info(
+ "Actual rows number: {}, expected: {}", actualBeamRows.size(), expectedBeamRows.size());
+
+ assertThat(
+ "Comparing expected rows with actual rows",
+ actualBeamRows,
+ containsInAnyOrder(expectedBeamRows.toArray()));
+ assertEquals(
+ "Checking there is no duplication", expectedBeamRows.size(), actualBeamRows.size());
+ }
+
+ @Test
+ public void testLoadWithFixedShards() throws IOException, InterruptedException {
+ runStreaming(5, false);
+ }
+
+ @Test
+ public void testLoadWithAutoShardingAndCopyJobs() throws IOException, InterruptedException {
+ runStreaming(0, true);
+ }
+
+ @Test
+ public void testDynamicDestinationsWithFixedShards() throws IOException, InterruptedException {
+ runStreamingToDynamicDestinations(6, false);
+ }
+
+ @Test
+ public void testDynamicDestinationsWithAutoShardingAndCopyJobs()
+ throws IOException, InterruptedException {
+ runStreamingToDynamicDestinations(0, true);
+ }
+
+ private void runStreamingToDynamicDestinations(int numFileShards, boolean useCopyJobs)
+ throws IOException, InterruptedException {
+ TestPipelineOptions opts = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class);
+ opts.setTempLocation(opts.getTempRoot());
+ Pipeline p = Pipeline.create(opts);
+ // Only run the most relevant test cases on Dataflow. Testing this dimension on DirectRunner is
+ // sufficient
+ if (p.getOptions().getRunner().getName().contains("DataflowRunner")) {
+ assumeTrue("Skipping in favor of more relevant test case", useInputSchema);
+ // Need to manually enable streaming engine for legacy dataflow runner
+ ExperimentalOptions.addExperiment(
+ p.getOptions().as(ExperimentalOptions.class), GcpOptions.STREAMING_ENGINE_EXPERIMENT);
+ }
+
+ List allFields = Arrays.asList(FIELDS);
+ List subFields0 = new ArrayList<>(allFields.subList(0, 4));
+ List subFields1 = new ArrayList<>(allFields.subList(4, 8));
+ List subFields2 = new ArrayList<>(allFields.subList(8, 11));
+ TableSchema table0Schema = makeTableSchemaFromTypes(subFields0);
+ TableSchema table1Schema = makeTableSchemaFromTypes(subFields1);
+ TableSchema table2Schema = makeTableSchemaFromTypes(subFields2);
+ String table0Id = maybeCreateTable(table0Schema, "-0");
+ String table1Id = maybeCreateTable(table1Schema, "-1");
+ String table2Id = maybeCreateTable(table2Schema, "-2");
+ GenerateRowFunc generateRowFunc0 = new GenerateRowFunc(subFields0);
+ GenerateRowFunc generateRowFunc1 = new GenerateRowFunc(subFields1);
+ GenerateRowFunc generateRowFunc2 = new GenerateRowFunc(subFields2);
+
+ String tablePrefix = table0Id.substring(0, table0Id.length() - 2);
+
+ // set up and build pipeline
+ Instant start = new Instant(0);
+ PCollection instants =
+ p.apply(
+ "Generate Instants",
+ PeriodicImpulse.create()
+ .startAt(start)
+ .stopAt(start.plus(Duration.standardSeconds(TOTAL_N - 1)))
+ .withInterval(Duration.standardSeconds(1))
+ .catchUpToNow(false));
+ PCollection longs =
+ instants.apply(
+ "Create TableRows",
+ MapElements.into(TypeDescriptors.longs()).via(instant -> instant.getMillis() / 1000));
+ // build write transform
+ Write write =
+ BigQueryIO.write()
+ .to(
+ new TestDynamicDest(
+ tablePrefix, subFields0, subFields1, subFields2, useInputSchema))
+ .withFormatFunction(
+ id -> {
+ long dest = id % 3;
+ TableRow row;
+ if (dest == 0) {
+ row = generateRowFunc0.apply(id);
+ } else if (dest == 1) {
+ row = generateRowFunc1.apply(id);
+ } else {
+ row = generateRowFunc2.apply(id);
+ }
+ return row;
+ })
+ .withMethod(Write.Method.FILE_LOADS)
+ .withTriggeringFrequency(Duration.standardSeconds(10));
+ if (useCopyJobs) {
+ write = write.withMaxBytesPerPartition(150);
+ }
+ if (useInputSchema) {
+ // we're creating the table with the input schema
+ write =
+ write
+ .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
+ .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE);
+ } else {
+ // table already exists with a schema, no need to create it
+ write =
+ write
+ .withCreateDisposition(CreateDisposition.CREATE_NEVER)
+ .withWriteDisposition(WriteDisposition.WRITE_APPEND);
+ }
+ write = numFileShards == 0 ? write.withAutoSharding() : write.withNumFileShards(numFileShards);
+
+ longs.apply("Stream loads to dynamic destinations", write);
+ p.run().waitUntilFinish();
+
+ List expectedRows0 = new ArrayList<>();
+ List expectedRows1 = new ArrayList<>();
+ List expectedRows2 = new ArrayList<>();
+ for (long i = 0; i < TOTAL_N; i++) {
+ long dest = i % 3;
+ if (dest == 0) {
+ expectedRows0.add(generateRowFunc0.apply(i));
+ } else if (dest == 1) {
+ expectedRows1.add(generateRowFunc1.apply(i));
+ } else {
+ expectedRows2.add(generateRowFunc2.apply(i));
+ }
+ }
+ // Perform checks
+ checkRowCompleteness(table0Id, makeTableSchemaFromTypes(subFields0), expectedRows0);
+ checkRowCompleteness(table1Id, makeTableSchemaFromTypes(subFields1), expectedRows1);
+ checkRowCompleteness(table2Id, makeTableSchemaFromTypes(subFields2), expectedRows2);
+ }
+
+ static class TestDynamicDest extends DynamicDestinations {
+ String tablePrefix;
+ List table0Fields;
+ List table1Fields;
+ List table2Fields;
+ boolean useInputSchema;
+
+ public TestDynamicDest(
+ String tablePrefix,
+ List table0Fields,
+ List table1Fields,
+ List table2Fields,
+ boolean useInputSchema) {
+ this.tablePrefix = tablePrefix;
+ this.table0Fields = table0Fields;
+ this.table1Fields = table1Fields;
+ this.table2Fields = table2Fields;
+ this.useInputSchema = useInputSchema;
+ }
+
+ @Override
+ public Long getDestination(@Nullable ValueInSingleWindow element) {
+ return element.getValue() % 3;
+ }
+
+ @Override
+ public TableDestination getTable(Long destination) {
+ return new TableDestination(tablePrefix + "-" + destination, null);
+ }
+
+ @Override
+ public @Nullable TableSchema getSchema(Long destination) {
+ if (!useInputSchema) {
+ return null;
+ }
+ List fields;
+ if (destination == 0) {
+ fields = table0Fields;
+ } else if (destination == 1) {
+ fields = table1Fields;
+ } else {
+ fields = table2Fields;
+ }
+ List tableFields =
+ fields.stream()
+ .map(name -> new TableFieldSchema().setName(name).setType(name).setMode("REQUIRED"))
+ .collect(Collectors.toList());
+ // we attach an ID to each row in addition to the existing schema fields
+ tableFields.add(
+ 0, new TableFieldSchema().setName("id").setType("INTEGER").setMode("REQUIRED"));
+ return new TableSchema().setFields(tableFields);
+ }
+ }
+}
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java
index df085bcedec3..54c636bde5fe 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProviderTest.java
@@ -48,9 +48,11 @@
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.MapElements;
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.sdk.values.TypeDescriptors;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@@ -211,7 +213,13 @@ public void testInputElementCount() throws Exception {
public void testFailedRows() throws Exception {
String tableSpec = "project:dataset.write_with_fail";
BigQueryStorageWriteApiSchemaTransformConfiguration config =
- BigQueryStorageWriteApiSchemaTransformConfiguration.builder().setTable(tableSpec).build();
+ BigQueryStorageWriteApiSchemaTransformConfiguration.builder()
+ .setTable(tableSpec)
+ .setErrorHandling(
+ BigQueryStorageWriteApiSchemaTransformConfiguration.ErrorHandling.builder()
+ .setOutput("FailedRows")
+ .build())
+ .build();
String failValue = "fail_me";
@@ -234,7 +242,15 @@ public void testFailedRows() throws Exception {
fakeDatasetService.setShouldFailRow(shouldFailRow);
PCollectionRowTuple result = runWithConfig(config, totalRows);
- PCollection failedRows = result.get("FailedRows");
+ PCollection failedRows =
+ result
+ .get("FailedRows")
+ .apply(
+ "ExtractFailedRows",
+ MapElements.into(TypeDescriptors.rows())
+ .via((rowAndError) -> rowAndError.getValue("failed_row")))
+ .setRowSchema(SCHEMA);
+ ;
PAssert.that(failedRows).containsInAnyOrder(expectedFailedRows);
p.run().waitUntilFinish();
@@ -250,7 +266,13 @@ public void testFailedRows() throws Exception {
public void testErrorCount() throws Exception {
String tableSpec = "project:dataset.error_count";
BigQueryStorageWriteApiSchemaTransformConfiguration config =
- BigQueryStorageWriteApiSchemaTransformConfiguration.builder().setTable(tableSpec).build();
+ BigQueryStorageWriteApiSchemaTransformConfiguration.builder()
+ .setTable(tableSpec)
+ .setErrorHandling(
+ BigQueryStorageWriteApiSchemaTransformConfiguration.ErrorHandling.builder()
+ .setOutput("FailedRows")
+ .build())
+ .build();
Function shouldFailRow =
(Function & Serializable) tr -> tr.get("name").equals("a");
diff --git a/sdks/java/testing/tpcds/build.gradle b/sdks/java/testing/tpcds/build.gradle
index 87d7ed11407a..387dda898247 100644
--- a/sdks/java/testing/tpcds/build.gradle
+++ b/sdks/java/testing/tpcds/build.gradle
@@ -126,6 +126,7 @@ task run(type: JavaExec) {
tpcdsArgsList.add("--sparkMaster=local[4]")
// Dataset runner only
systemProperty "spark.sql.shuffle.partitions", "4"
+ systemProperty "spark.sql.adaptive.enabled", "false" // high overhead for complex queries
}
mainClass = "org.apache.beam.sdk.tpcds.BeamTpcds"
diff --git a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java
index 3878c42affe0..f52fdfed710d 100644
--- a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java
+++ b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java
@@ -58,7 +58,7 @@ public class TransformServiceLauncher {
private Map environmentVariables = new HashMap<>();
// Amount of time (in milliseconds) to wait till the Docker Compose starts up.
- private static final int DEFAULT_START_WAIT_TIME = 25000;
+ private static final int DEFAULT_START_WAIT_TIME = 50000;
private static final int STATUS_LOGGER_WAIT_TIME = 3000;
@SuppressWarnings("argument")
diff --git a/sdks/python/.pylintrc b/sdks/python/.pylintrc
index 38913a326a4a..250932e79812 100644
--- a/sdks/python/.pylintrc
+++ b/sdks/python/.pylintrc
@@ -152,7 +152,6 @@ disable =
unnecessary-lambda-assignment,
unnecessary-pass,
unneeded-not,
- use-implicit-booleaness-not-comparison, #TODO(https://github.com/apache/beam/issues/28244) Enable and fix warnings
used-before-assignment,
unsubscriptable-object,
unsupported-binary-operation,
diff --git a/sdks/python/apache_beam/dataframe/frame_base.py b/sdks/python/apache_beam/dataframe/frame_base.py
index 24497f1de069..1da12ececff6 100644
--- a/sdks/python/apache_beam/dataframe/frame_base.py
+++ b/sdks/python/apache_beam/dataframe/frame_base.py
@@ -475,7 +475,7 @@ def wrapper(self, inplace=False, **kwargs):
return wrapper
-def args_to_kwargs(base_type):
+def args_to_kwargs(base_type, removed_method=False, removed_args=None):
"""Convert all args to kwargs before calling the decorated function.
When applied to a function, this decorator creates a new function
@@ -484,18 +484,47 @@ def args_to_kwargs(base_type):
determine the name to use for arguments that are converted to keyword
arguments.
- For internal use only. No backwards compatibility guarantees."""
+ For internal use only. No backwards compatibility guarantees.
+
+ Args:
+ base_type: The pandas type of the method that this is trying to replicate.
+ removed_method: Whether this method has been removed in the running
+ Pandas version.
+ removed_args: If not empty, which arguments have been dropped in the
+ running Pandas version.
+ """
def wrap(func):
- arg_names = getfullargspec(unwrap(getattr(base_type, func.__name__))).args
+ if removed_method:
+ # Do no processing, let Beam function itself raise the error if called.
+ return func
+
+ removed_arg_names = removed_args if removed_args is not None else []
+
+ base_arg_spec = getfullargspec(unwrap(getattr(base_type, func.__name__)))
+ base_arg_names = base_arg_spec.args
+ # Some arguments are keyword only and we still want to check against those.
+ all_possible_base_arg_names = base_arg_names + base_arg_spec.kwonlyargs
+ beam_arg_names = getfullargspec(func).args
+
+ if not_found := (set(beam_arg_names) - set(all_possible_base_arg_names) -
+ set(removed_arg_names)):
+ raise TypeError(
+ f"Beam definition of {func.__name__} has arguments that are not found"
+ f" in the base version of the function: {not_found}")
@functools.wraps(func)
def wrapper(*args, **kwargs):
- for name, value in zip(arg_names, args):
+ for name, value in zip(base_arg_names, args):
if name in kwargs:
raise TypeError(
"%s() got multiple values for argument '%s'" %
(func.__name__, name))
kwargs[name] = value
+ # Still have to populate these for the Beam function signature.
+ if removed_args:
+ for name in removed_args:
+ if not name in kwargs:
+ kwargs[name] = None
return func(**kwargs)
return wrapper
@@ -524,14 +553,22 @@ def wrapper(*args, **kwargs):
f"**{BEAM_SPECIFIC!r}** for details.")
-def with_docs_from(base_type, name=None):
+def with_docs_from(base_type, name=None, removed_method=False):
"""Decorator that updates the documentation from the wrapped function to
duplicate the documentation from the identically-named method in `base_type`.
Any docstring on the original function will be included in the new function
under a "Differences from pandas" heading.
+
+ removed_method used in cases where a method has been removed in a later
+ version of Pandas.
"""
def wrap(func):
+ if removed_method:
+ func.__doc__ = (
+ "This method has been removed in the current version of Pandas.")
+ return func
+
fn_name = name or func.__name__
orig_doc = getattr(base_type, fn_name).__doc__
if orig_doc is None:
@@ -588,15 +625,26 @@ def format_section(header):
return wrap
-def populate_defaults(base_type):
+def populate_defaults(base_type, removed_method=False, removed_args=None):
"""Populate default values for keyword arguments in decorated function.
When applied to a function, this decorator creates a new function
with default values for all keyword arguments, based on the default values
for the identically-named method on `base_type`.
- For internal use only. No backwards compatibility guarantees."""
+ For internal use only. No backwards compatibility guarantees.
+
+ Args:
+ base_type: The pandas type of the method that this is trying to replicate.
+ removed_method: Whether this method has been removed in the running
+ Pandas version.
+ removed_args: If not empty, which arguments have been dropped in the
+ running Pandas version.
+ """
def wrap(func):
+ if removed_method:
+ return func
+
base_argspec = getfullargspec(unwrap(getattr(base_type, func.__name__)))
if not base_argspec.defaults:
return func
@@ -613,6 +661,8 @@ def wrap(func):
defaults_to_populate = set(
func_argspec.args[:num_non_defaults]).intersection(
arg_to_default.keys())
+ if removed_args:
+ defaults_to_populate -= set(removed_args)
@functools.wraps(func)
def wrapper(**kwargs):
diff --git a/sdks/python/apache_beam/dataframe/frame_base_test.py b/sdks/python/apache_beam/dataframe/frame_base_test.py
index 82d5b65e1a49..2d16d02ba1ea 100644
--- a/sdks/python/apache_beam/dataframe/frame_base_test.py
+++ b/sdks/python/apache_beam/dataframe/frame_base_test.py
@@ -93,14 +93,27 @@ class Base(object):
def func(self, a=1, b=2, c=3):
pass
+ def func_removed_args(self, a):
+ pass
+
class Proxy(object):
@frame_base.args_to_kwargs(Base)
@frame_base.populate_defaults(Base)
def func(self, a, c=1000, **kwargs):
return dict(kwargs, a=a, c=c)
+ @frame_base.args_to_kwargs(Base, removed_method=True)
+ @frame_base.populate_defaults(Base, removed_method=True)
+ def func_removed_method(self, a, **kwargs):
+ return dict(kwargs, a=a)
+
+ @frame_base.args_to_kwargs(Base, removed_args=['c'])
+ @frame_base.populate_defaults(Base, removed_args=['c'])
+ def func_removed_args(self, a, c, **kwargs):
+ return dict(kwargs, a=a)
+
proxy = Proxy()
- # pylint: disable=too-many-function-args
+ # pylint: disable=too-many-function-args,no-value-for-parameter
self.assertEqual(proxy.func(), {'a': 1, 'c': 1000})
self.assertEqual(proxy.func(100), {'a': 100, 'c': 1000})
self.assertEqual(proxy.func(2, 4, 6), {'a': 2, 'b': 4, 'c': 6})
@@ -108,6 +121,14 @@ def func(self, a, c=1000, **kwargs):
self.assertEqual(proxy.func(c=6, a=2), {'a': 2, 'c': 6})
self.assertEqual(proxy.func(c=6), {'a': 1, 'c': 6})
+ with self.assertRaises(TypeError): # missing 1 required positional argument
+ proxy.func_removed_method()
+ self.assertEqual(proxy.func_removed_method(12, c=100), {'a': 12, 'c': 100})
+
+ with self.assertRaises(TypeError): # missing 1 required positional argument
+ proxy.func_removed_args()
+ self.assertEqual(proxy.func_removed_args(12, d=100), {'a': 12, 'd': 100})
+
if __name__ == '__main__':
unittest.main()
diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py
index 3020eecbaeb5..e2390bda28be 100644
--- a/sdks/python/apache_beam/dataframe/frames.py
+++ b/sdks/python/apache_beam/dataframe/frames.py
@@ -116,7 +116,6 @@ def wrapper(self, *args, **kwargs):
'quantile',
'describe',
'sem',
- 'mad',
'skew',
'kurt',
'kurtosis',
@@ -126,6 +125,10 @@ def wrapper(self, *args, **kwargs):
'cov',
'nunique',
]
+# mad was removed in Pandas 2.0.
+if PD_VERSION < (2, 0):
+ UNLIFTABLE_AGGREGATIONS.append('mad')
+
ALL_AGGREGATIONS = (
LIFTABLE_AGGREGATIONS + LIFTABLE_WITH_SUM_AGGREGATIONS +
UNLIFTABLE_AGGREGATIONS)
@@ -911,8 +914,10 @@ def sort_index(self, axis, **kwargs):
))
@frame_base.with_docs_from(pd.DataFrame)
- @frame_base.args_to_kwargs(pd.DataFrame)
- @frame_base.populate_defaults(pd.DataFrame)
+ @frame_base.args_to_kwargs(
+ pd.DataFrame, removed_args=["errors"] if PD_VERSION >= (2, 0) else None)
+ @frame_base.populate_defaults(
+ pd.DataFrame, removed_args=["errors"] if PD_VERSION >= (2, 0) else None)
@frame_base.maybe_inplace
def where(self, cond, other, errors, **kwargs):
"""where is not parallelizable when ``errors="ignore"`` is specified."""
@@ -934,16 +939,19 @@ def where(self, cond, other, errors, **kwargs):
else:
actual_args['other'] = other
- if errors == "ignore":
- # We need all data in order to ignore errors and propagate the original
- # data.
- requires = partitionings.Singleton(
- reason=(
- f"where(errors={errors!r}) is currently not parallelizable, "
- "because all data must be collected on one node to determine if "
- "the original data should be propagated instead."))
+ # For Pandas 2.0, errors was removed as an argument.
+ if PD_VERSION < (2, 0):
+ if "errors" in kwargs and kwargs['errors'] == "ignore":
+ # We need all data in order to ignore errors and propagate the original
+ # data.
+ requires = partitionings.Singleton(
+ reason=(
+ f"where(errors={kwargs['errors']!r}) is currently not "
+ "parallelizable, because all data must be collected on one "
+ "node to determine if the original data should be propagated "
+ "instead."))
- actual_args['errors'] = errors
+ actual_args['errors'] = kwargs['errors'] if 'errors' in kwargs else None
def where_execution(df, *args):
runtime_values = {
@@ -1333,12 +1341,14 @@ def keys(self):
frame_base.wont_implement_method(
pd.Series, 'shape', reason="non-deferred-result"))
- @frame_base.with_docs_from(pd.Series)
- @frame_base.args_to_kwargs(pd.Series)
- @frame_base.populate_defaults(pd.Series)
+ @frame_base.with_docs_from(pd.Series, removed_method=PD_VERSION >= (2, 0))
+ @frame_base.args_to_kwargs(pd.Series, removed_method=PD_VERSION >= (2, 0))
+ @frame_base.populate_defaults(pd.Series, removed_method=PD_VERSION >= (2, 0))
def append(self, to_append, ignore_index, verify_integrity, **kwargs):
"""``ignore_index=True`` is not supported, because it requires generating an
order-sensitive index."""
+ if PD_VERSION >= (2, 0):
+ raise frame_base.WontImplementError('append() was removed in Pandas 2.0.')
if not isinstance(to_append, DeferredSeries):
raise frame_base.WontImplementError(
"append() only accepts DeferredSeries instances, received " +
@@ -1600,14 +1610,11 @@ def mean(self, skipna, **kwargs):
return self.sum(skipna=skipna, **kwargs) / size
@frame_base.with_docs_from(pd.Series)
- @frame_base.args_to_kwargs(pd.Series)
- @frame_base.populate_defaults(pd.Series)
+ @frame_base.args_to_kwargs(
+ pd.Series, removed_args=["level"] if PD_VERSION >= (2, 0) else None)
+ @frame_base.populate_defaults(
+ pd.Series, removed_args=["level"] if PD_VERSION >= (2, 0) else None)
def var(self, axis, skipna, level, ddof, **kwargs):
- """Per-level aggregation is not yet supported
- (https://github.com/apache/beam/issues/21829). Only the default,
- ``level=None``, is allowed."""
- if level is not None:
- raise NotImplementedError("per-level aggregation")
if skipna is None or skipna:
self = self.dropna() # pylint: disable=self-cls-assignment
@@ -1675,11 +1682,11 @@ def corr(self, other, method, min_periods):
requires_partition_by=partitionings.Singleton(reason=reason)))
@frame_base.with_docs_from(pd.Series)
- @frame_base.args_to_kwargs(pd.Series)
- @frame_base.populate_defaults(pd.Series)
+ @frame_base.args_to_kwargs(
+ pd.Series, removed_args=["level"] if PD_VERSION >= (2, 0) else None)
+ @frame_base.populate_defaults(
+ pd.Series, removed_args=["level"] if PD_VERSION >= (2, 0) else None)
def skew(self, axis, skipna, level, numeric_only, **kwargs):
- if level is not None:
- raise NotImplementedError("per-level aggregation")
if skipna is None or skipna:
self = self.dropna() # pylint: disable=self-cls-assignment
# See the online, numerically stable formulae at
@@ -1739,11 +1746,11 @@ def combine_moments(data):
requires_partition_by=partitionings.Singleton()))
@frame_base.with_docs_from(pd.Series)
- @frame_base.args_to_kwargs(pd.Series)
- @frame_base.populate_defaults(pd.Series)
+ @frame_base.args_to_kwargs(
+ pd.Series, removed_args=["level"] if PD_VERSION >= (2, 0) else None)
+ @frame_base.populate_defaults(
+ pd.Series, removed_args=["level"] if PD_VERSION >= (2, 0) else None)
def kurtosis(self, axis, skipna, level, numeric_only, **kwargs):
- if level is not None:
- raise NotImplementedError("per-level aggregation")
if skipna is None or skipna:
self = self.dropna() # pylint: disable=self-cls-assignment
@@ -2092,7 +2099,9 @@ def axes(self):
sum = _agg_method(pd.Series, 'sum')
median = _agg_method(pd.Series, 'median')
sem = _agg_method(pd.Series, 'sem')
- mad = _agg_method(pd.Series, 'mad')
+ # mad was removed in Pandas 2.0.
+ if PD_VERSION < (2, 0):
+ mad = _agg_method(pd.Series, 'mad')
argmax = frame_base.wont_implement_method(
pd.Series, 'argmax', reason='order-sensitive')
@@ -2571,7 +2580,8 @@ def align(self, other, join, axis, copy, level, method, **kwargs):
if kwargs:
raise NotImplementedError('align(%s)' % ', '.join(kwargs.keys()))
- if level is not None:
+ # In Pandas 2.0, all aggregations lost the level keyword.
+ if PD_VERSION < (2, 0) and level is not None:
# Could probably get by partitioning on the used levels.
requires_partition_by = partitionings.Singleton(reason=(
f"align(level={level}) is not currently parallelizable. Only "
@@ -2588,12 +2598,15 @@ def align(self, other, join, axis, copy, level, method, **kwargs):
requires_partition_by=requires_partition_by,
preserves_partition_by=partitionings.Arbitrary()))
- @frame_base.with_docs_from(pd.DataFrame)
- @frame_base.args_to_kwargs(pd.DataFrame)
- @frame_base.populate_defaults(pd.DataFrame)
+ @frame_base.with_docs_from(pd.DataFrame, removed_method=PD_VERSION >= (2, 0))
+ @frame_base.args_to_kwargs(pd.DataFrame, removed_method=PD_VERSION >= (2, 0))
+ @frame_base.populate_defaults(pd.DataFrame,
+ removed_method=PD_VERSION >= (2, 0))
def append(self, other, ignore_index, verify_integrity, sort, **kwargs):
"""``ignore_index=True`` is not supported, because it requires generating an
order-sensitive index."""
+ if PD_VERSION >= (2, 0):
+ raise frame_base.WontImplementError('append() was removed in Pandas 2.0.')
if not isinstance(other, DeferredDataFrame):
raise frame_base.WontImplementError(
"append() only accepts DeferredDataFrame instances, received " +
@@ -3914,10 +3927,12 @@ def pivot_helper(df):
std = _agg_method(pd.DataFrame, 'std')
var = _agg_method(pd.DataFrame, 'var')
sem = _agg_method(pd.DataFrame, 'sem')
- mad = _agg_method(pd.DataFrame, 'mad')
skew = _agg_method(pd.DataFrame, 'skew')
kurt = _agg_method(pd.DataFrame, 'kurt')
kurtosis = _agg_method(pd.DataFrame, 'kurtosis')
+ # mad was removed in Pandas 2.0.
+ if PD_VERSION < (2, 0):
+ mad = _agg_method(pd.DataFrame, 'mad')
take = frame_base.wont_implement_method(pd.DataFrame, 'take',
reason='deprecated')
@@ -4670,7 +4685,10 @@ def _is_unliftable(agg_func):
return _check_str_or_np_builtin(agg_func, UNLIFTABLE_AGGREGATIONS)
NUMERIC_AGGREGATIONS = ['max', 'min', 'prod', 'sum', 'mean', 'median', 'std',
- 'var', 'sem', 'mad', 'skew', 'kurt', 'kurtosis']
+ 'var', 'sem', 'skew', 'kurt', 'kurtosis']
+# mad was removed in Pandas 2.0.
+if PD_VERSION < (2, 0):
+ NUMERIC_AGGREGATIONS.append('mad')
def _is_numeric(agg_func):
return _check_str_or_np_builtin(agg_func, NUMERIC_AGGREGATIONS)
@@ -4698,7 +4716,6 @@ class _DeferredGroupByCols(frame_base.DeferredFrame):
idxmax = frame_base._elementwise_method('idxmax', base=DataFrameGroupBy)
idxmin = frame_base._elementwise_method('idxmin', base=DataFrameGroupBy)
last = frame_base._elementwise_method('last', base=DataFrameGroupBy)
- mad = frame_base._elementwise_method('mad', base=DataFrameGroupBy)
max = frame_base._elementwise_method('max', base=DataFrameGroupBy)
mean = frame_base._elementwise_method('mean', base=DataFrameGroupBy)
median = frame_base._elementwise_method('median', base=DataFrameGroupBy)
@@ -4717,8 +4734,11 @@ class _DeferredGroupByCols(frame_base.DeferredFrame):
DataFrameGroupBy, 'tail', explanation=_PEEK_METHOD_EXPLANATION)
take = frame_base.wont_implement_method(
DataFrameGroupBy, 'take', reason='deprecated')
- tshift = frame_base._elementwise_method('tshift', base=DataFrameGroupBy)
var = frame_base._elementwise_method('var', base=DataFrameGroupBy)
+ # These already deprecated methods were removed in Pandas 2.0
+ if PD_VERSION < (2, 0):
+ mad = frame_base._elementwise_method('mad', base=DataFrameGroupBy)
+ tshift = frame_base._elementwise_method('tshift', base=DataFrameGroupBy)
@property # type: ignore
@frame_base.with_docs_from(DataFrameGroupBy)
@@ -5376,6 +5396,7 @@ def func(df, *args, **kwargs):
name,
frame_base._elementwise_method(name, restrictions={'level': None},
base=pd.Series))
+
if hasattr(pd.DataFrame, name):
setattr(
DeferredDataFrame,
diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py
index a2a8ef75f885..4e59d1da5de4 100644
--- a/sdks/python/apache_beam/dataframe/frames_test.py
+++ b/sdks/python/apache_beam/dataframe/frames_test.py
@@ -797,6 +797,7 @@ def test_loc(self):
self._run_test(lambda df: df.C.loc[df.A > 10], df)
self._run_test(lambda df, s: df.loc[s.loc[1:3]], df, pd.Series(dates))
+ @unittest.skipIf(PD_VERSION >= (2, 0), 'append removed in Pandas 2.0')
def test_append_sort(self):
# yapf: disable
df1 = pd.DataFrame({'int': [1, 2, 3], 'str': ['a', 'b', 'c']},
@@ -985,6 +986,7 @@ def test_series_fillna_series_as_value(self):
self._run_test(lambda df, df2: df.A.fillna(df2.A), df, df2)
+ @unittest.skipIf(PD_VERSION >= (2, 0), 'append removed in Pandas 2.0')
def test_append_verify_integrity(self):
df1 = pd.DataFrame({'A': range(10), 'B': range(10)}, index=range(10))
df2 = pd.DataFrame({'A': range(10), 'B': range(10)}, index=range(9, 19))
@@ -1946,6 +1948,12 @@ def test_groupby_multiindex_keep_nans(self):
lambda df: df.groupby(['foo', 'bar'], dropna=False).sum(), GROUPBY_DF)
+NONPARALLEL_METHODS = ['quantile', 'describe', 'median', 'sem']
+# mad was removed in pandas 2
+if PD_VERSION < (2, 0):
+ NONPARALLEL_METHODS.append('mad')
+
+
class AggregationTest(_AbstractFrameTest):
"""Tests for global aggregation methods on DataFrame/Series."""
@@ -1955,7 +1963,7 @@ class AggregationTest(_AbstractFrameTest):
def test_series_agg(self, agg_method):
s = pd.Series(list(range(16)))
- nonparallel = agg_method in ('quantile', 'describe', 'median', 'sem', 'mad')
+ nonparallel = agg_method in NONPARALLEL_METHODS
# TODO(https://github.com/apache/beam/issues/20926): max and min produce
# the wrong proxy
@@ -1974,7 +1982,7 @@ def test_series_agg(self, agg_method):
def test_series_agg_method(self, agg_method):
s = pd.Series(list(range(16)))
- nonparallel = agg_method in ('quantile', 'describe', 'median', 'sem', 'mad')
+ nonparallel = agg_method in NONPARALLEL_METHODS
# TODO(https://github.com/apache/beam/issues/20926): max and min produce
# the wrong proxy
@@ -1990,7 +1998,7 @@ def test_series_agg_method(self, agg_method):
def test_dataframe_agg(self, agg_method):
df = pd.DataFrame({'A': [1, 2, 3, 4], 'B': [2, 3, 5, 7]})
- nonparallel = agg_method in ('quantile', 'describe', 'median', 'sem', 'mad')
+ nonparallel = agg_method in NONPARALLEL_METHODS
# TODO(https://github.com/apache/beam/issues/20926): max and min produce
# the wrong proxy
@@ -2007,7 +2015,7 @@ def test_dataframe_agg(self, agg_method):
def test_dataframe_agg_method(self, agg_method):
df = pd.DataFrame({'A': [1, 2, 3, 4], 'B': [2, 3, 5, 7]})
- nonparallel = agg_method in ('quantile', 'describe', 'median', 'sem', 'mad')
+ nonparallel = agg_method in NONPARALLEL_METHODS
# TODO(https://github.com/apache/beam/issues/20926): max and min produce
# the wrong proxy
@@ -2036,6 +2044,7 @@ def test_dataframe_agg_modes(self):
self._run_test(lambda df: df.agg({'A': ['sum', 'mean']}), df)
self._run_test(lambda df: df.agg({'A': ['sum', 'mean'], 'B': 'min'}), df)
+ @unittest.skipIf(PD_VERSION >= (2, 0), "level argument removed in Pandas 2")
def test_series_agg_level(self):
self._run_test(
lambda df: df.set_index(['group', 'foo']).bar.count(level=0),
@@ -2059,6 +2068,7 @@ def test_series_agg_level(self):
lambda df: df.set_index(['group', 'foo']).bar.median(level=1),
GROUPBY_DF)
+ @unittest.skipIf(PD_VERSION >= (2, 0), "level argument removed in Pandas 2")
def test_dataframe_agg_level(self):
self._run_test(
lambda df: df.set_index(['group', 'foo']).count(level=0), GROUPBY_DF)
@@ -2226,6 +2236,7 @@ def test_df_agg_method_invalid_kwarg_raises(self):
self._run_error_test(
lambda df: df.median(min_count=3, numeric_only=True), GROUPBY_DF)
+ @unittest.skipIf(PD_VERSION >= (2, 0), "level argument removed in Pandas 2")
def test_agg_min_count(self):
df = pd.DataFrame({
'good': [1, 2, 3, np.nan],
diff --git a/sdks/python/apache_beam/examples/inference/README.md b/sdks/python/apache_beam/examples/inference/README.md
index e66e1ce471d4..19262dead586 100644
--- a/sdks/python/apache_beam/examples/inference/README.md
+++ b/sdks/python/apache_beam/examples/inference/README.md
@@ -815,4 +815,72 @@ Each line has data separated by a semicolon ";".
The first item is the input sentence. The model masks the last word and tries to predict it;
the second item is the word that the model predicts for the mask.
+---
+## Image classifcation with Vertex AI
+
+[`vertex_ai_image_classification.py`](./vertex_ai_image_classification.py) contains an implementation for a RunInference pipeline that performs image classification using a model hosted on Vertex AI (based on https://cloud.google.com/vertex-ai/docs/tutorials/image-recognition-custom).
+
+The pipeline reads image urls, performs basic preprocessing to convert them into a List of floats, passes the masked sentence to the Vertex AI implementation of RunInference, and then writes the predictions to a text file.
+
+### Dataset and model for image classification
+
+To use this transform, you need a dataset and model hosted on Vertex AI for image classification.
+
+1. Train a model by following the tutorial at https://cloud.google.com/vertex-ai/docs/tutorials/image-recognition-custom
+2. Create a file named `IMAGE_FILE_NAMES.txt` that contains the absolute paths of each of the images in `IMAGES_DIR` that you want to use to run image classification. The path to the file can be different types of URIs such as your local file system, an AWS S3 bucket, or a GCP Cloud Storage bucket. For example:
+```
+/absolute/path/to/image1.jpg
+/absolute/path/to/image2.jpg
+```
+
+### Running `vertex_ai_image_classification.py`
+
+To run the image classification pipeline locally, use the following command:
+```sh
+python -m apache_beam.examples.inference.vertex_ai_image_classification \
+ --endpoint_id '' \
+ --endpoint_project '' \
+ --endpoint_region '' \
+ --input 'path/to/IMAGE_FILE_NAMES.txt' \
+ --output 'path/to/output/file.txt'
+```
+
+This writes the output to the output file with contents like:
+```
+path/to/my/image: tulips (90)
+path/to/my/image2: dandelions (78)
+...
+```
+Each line represents a prediction of the flower type along with the confidence in that prediction.
+
+---
+## Text classifcation with a Vertex AI LLM
+
+[`vertex_ai_llm_text_classification.py`](./vertex_ai_llm_text_classification.py) contains an implementation for a RunInference pipeline that performs image classification using a model hosted on Vertex AI (based on https://cloud.google.com/vertex-ai/docs/tutorials/image-recognition-custom).
+
+The pipeline reads image urls, performs basic preprocessing to convert them into a List of floats, passes the masked sentence to the Vertex AI implementation of RunInference, and then writes the predictions to a text file.
+
+### Dataset and model for image classification
+
+To use this transform, you need a model hosted on Vertex AI for text classification.
+You can get this by tuning the text-bison model following the instructions here -
+https://cloud.google.com/vertex-ai/docs/generative-ai/models/tune-models#create_a_model_tuning_job
+
+### Running `vertex_ai_llm_text_classification.py`
+
+To run the text classification pipeline locally, use the following command:
+```sh
+python -m apache_beam.examples.inference.vertex_ai_llm_text_classification \
+ --endpoint_id '' \
+ --endpoint_project '' \
+ --endpoint_region ''
+```
+
+This writes the output to the output file with contents like:
+```
+('What is 5+2?', PredictionResult(example={'prompt': 'What is 5+2?'}, inference={'content': '7', 'citationMetadata': {'citations': []}, 'safetyAttributes': {'blocked': False, 'scores': [], 'categories': []}}, model_id='6795590989097467904'))
+...
+```
+Each line represents a tuple containing the example, a [PredictionResult](https://beam.apache.org/releases/pydoc/2.40.0/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.PredictionResult)
+object with the response from the model in the inference field, and the endpoint id representing the model id.
---
\ No newline at end of file
diff --git a/sdks/python/apache_beam/examples/inference/vertex_ai_llm_text_classification.py b/sdks/python/apache_beam/examples/inference/vertex_ai_llm_text_classification.py
new file mode 100644
index 000000000000..e587ba87b91b
--- /dev/null
+++ b/sdks/python/apache_beam/examples/inference/vertex_ai_llm_text_classification.py
@@ -0,0 +1,135 @@
+#
+# 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.
+#
+
+""" A sample pipeline using the RunInference API to classify text using an LLM.
+This pipeline creates a set of prompts and sends it to a deployed Vertex AI
+model endpoint, then returns the predictions from the classifier model. The
+model can be generated by fine tuning the text-bison model or another similar
+model (see
+https://cloud.google.com/vertex-ai/docs/generative-ai/models/tune-models#supervised-fine-tuning)
+"""
+
+import argparse
+import logging
+
+import apache_beam as beam
+from apache_beam.ml.inference.base import KeyedModelHandler
+from apache_beam.ml.inference.base import RunInference
+from apache_beam.ml.inference.vertex_ai_inference import VertexAIModelHandlerJSON
+from apache_beam.options.pipeline_options import PipelineOptions
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.runners.runner import PipelineResult
+
+
+def parse_known_args(argv):
+ """Parses args for the workflow."""
+ parser = argparse.ArgumentParser()
+ parser.add_argument(
+ '--output',
+ dest='output',
+ type=str,
+ required=True,
+ help='Path to save output predictions.')
+ parser.add_argument(
+ '--endpoint_id',
+ dest='endpoint',
+ type=str,
+ required=True,
+ help='Vertex AI Endpoint resource ID to query (string).')
+ parser.add_argument(
+ '--endpoint_project', dest='project', required=True, help='GCP Project')
+ parser.add_argument(
+ '--endpoint_region',
+ dest='location',
+ type=str,
+ required=True,
+ help='GCP location for the Endpoint')
+ parser.add_argument(
+ '--endpoint_network',
+ dest='vpc_network',
+ type=str,
+ required=False,
+ help='GCP network the endpoint is peered to')
+ parser.add_argument(
+ '--experiment',
+ dest='experiment',
+ type=str,
+ required=False,
+ help='Vertex AI experiment label to apply to queries')
+ parser.add_argument(
+ '--private',
+ dest='private',
+ type=bool,
+ default=False,
+ help="True if the Vertex AI endpoint is a private endpoint")
+ return parser.parse_known_args(argv)
+
+
+def run(
+ argv=None, save_main_session=True, test_pipeline=None) -> PipelineResult:
+ """
+ Args:
+ argv: Command line arguments defined for this example.
+ save_main_session: Used for internal testing.
+ test_pipeline: Used for internal testing.
+ """
+ known_args, pipeline_args = parse_known_args(argv)
+ pipeline_options = PipelineOptions(pipeline_args)
+ pipeline_options.view_as(SetupOptions).save_main_session = save_main_session
+
+ model_handler = VertexAIModelHandlerJSON(
+ endpoint_id=known_args.endpoint,
+ project=known_args.project,
+ location=known_args.location,
+ experiment=known_args.experiment,
+ network=known_args.vpc_network,
+ private=known_args.private)
+
+ pipeline = test_pipeline
+ if not test_pipeline:
+ pipeline = beam.Pipeline(options=pipeline_options)
+
+ # For this example, use the default parameters from
+ # https://cloud.google.com/vertex-ai/docs/generative-ai/start/quickstarts/api-quickstart#parameter_definitions
+ parameters = {
+ "temperature": 0.2, "maxOutputTokens": 256, "topK": 40, "topP": 0.95
+ }
+ prompts = [
+ "What is 5+2?",
+ "Who is the president?",
+ "Write me a business plan for a cookie shop."
+ ]
+
+ read_prompts = pipeline | "Get prompt" >> beam.Create(prompts)
+ preprocess = read_prompts | "Format prompt" >> beam.Map(
+ lambda data: (data, {
+ "prompt": data
+ }))
+ predictions = preprocess | "RunInference" >> RunInference(
+ KeyedModelHandler(model_handler), inference_args=parameters)
+ _ = predictions | "PrintOutput" >> beam.Map(print)
+ _ = predictions | "WriteOutput" >> beam.io.WriteToText(
+ known_args.output, shard_name_template='', append_trailing_newlines=True)
+
+ result = pipeline.run()
+ result.wait_until_finish()
+ return result
+
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ run()
diff --git a/sdks/python/apache_beam/internal/dill_pickler.py b/sdks/python/apache_beam/internal/dill_pickler.py
index efa736c2a96f..8a0742642dfb 100644
--- a/sdks/python/apache_beam/internal/dill_pickler.py
+++ b/sdks/python/apache_beam/internal/dill_pickler.py
@@ -46,7 +46,7 @@
settings = {'dill_byref': None}
-if sys.version_info >= (3, 11) and dill.__version__ == "0.3.1.1":
+if sys.version_info >= (3, 10) and dill.__version__ == "0.3.1.1":
# Let's make dill 0.3.1.1 support Python 3.11.
# The following function is based on 'save_code' from 'dill'
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index afdd94740e9f..e092ad069ad0 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -417,6 +417,7 @@ def chain_after(result):
from apache_beam.transforms.util import ReshufflePerKey
from apache_beam.transforms.window import GlobalWindows
from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.schemas import schema_from_element_type
from apache_beam.utils import retry
from apache_beam.utils.annotations import deprecated
@@ -2148,6 +2149,7 @@ def expand(self, pcoll):
failed_rows=outputs[BigQueryWriteFn.FAILED_ROWS],
failed_rows_with_errors=outputs[
BigQueryWriteFn.FAILED_ROWS_WITH_ERRORS])
+
elif method_to_use == WriteToBigQuery.Method.FILE_LOADS:
if self._temp_file_format == bigquery_tools.FileFormat.AVRO:
if self.schema == SCHEMA_AUTODETECT:
@@ -2212,33 +2214,45 @@ def find_in_nested_dict(schema):
BigQueryBatchFileLoads.DESTINATION_FILE_PAIRS],
destination_copy_jobid_pairs=output[
BigQueryBatchFileLoads.DESTINATION_COPY_JOBID_PAIRS])
- else:
- # Storage Write API
+
+ elif method_to_use == WriteToBigQuery.Method.STORAGE_WRITE_API:
if self.schema is None:
- raise AttributeError(
- "A schema is required in order to prepare rows"
- "for writing with STORAGE_WRITE_API.")
- if callable(self.schema):
+ try:
+ schema = schema_from_element_type(pcoll.element_type)
+ is_rows = True
+ except TypeError as exn:
+ raise ValueError(
+ "A schema is required in order to prepare rows"
+ "for writing with STORAGE_WRITE_API.") from exn
+ elif callable(self.schema):
raise NotImplementedError(
"Writing to dynamic destinations is not"
"supported for this write method.")
elif isinstance(self.schema, vp.ValueProvider):
schema = self.schema.get()
+ is_rows = False
else:
schema = self.schema
+ is_rows = False
table = bigquery_tools.get_hashable_destination(self.table_reference)
# None type is not supported
triggering_frequency = self.triggering_frequency or 0
# SchemaTransform expects Beam Rows, so map to Rows first
+ if is_rows:
+ input_beam_rows = pcoll
+ else:
+ input_beam_rows = (
+ pcoll
+ | "Convert dict to Beam Row" >> beam.Map(
+ lambda row: bigquery_tools.beam_row_from_dict(row, schema)
+ ).with_output_types(
+ RowTypeConstraint.from_fields(
+ bigquery_tools.get_beam_typehints_from_tableschema(schema)))
+ )
output_beam_rows = (
- pcoll
- | "Convert dict to Beam Row" >>
- beam.Map(lambda row: bigquery_tools.beam_row_from_dict(row, schema)).
- with_output_types(
- RowTypeConstraint.from_fields(
- bigquery_tools.get_beam_typehints_from_tableschema(schema)))
- | "StorageWriteToBigQuery" >> StorageWriteToBigQuery(
+ input_beam_rows
+ | StorageWriteToBigQuery(
table=table,
create_disposition=self.create_disposition,
write_disposition=self.write_disposition,
@@ -2247,23 +2261,31 @@ def find_in_nested_dict(schema):
with_auto_sharding=self.with_auto_sharding,
expansion_service=self.expansion_service))
- # return back from Beam Rows to Python dict elements
- failed_rows = (
- output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS]
- | beam.Map(lambda row: row.as_dict()))
- failed_rows_with_errors = (
- output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS]
- | beam.Map(
- lambda row: {
- "error_message": row.error_message,
- "failed_row": row.failed_row.as_dict()
- }))
+ if is_rows:
+ failed_rows = output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS]
+ failed_rows_with_errors = output_beam_rows[
+ StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS]
+ else:
+ # return back from Beam Rows to Python dict elements
+ failed_rows = (
+ output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS]
+ | beam.Map(lambda row: row.as_dict()))
+ failed_rows_with_errors = (
+ output_beam_rows[StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS]
+ | beam.Map(
+ lambda row: {
+ "error_message": row.error_message,
+ "failed_row": row.failed_row.as_dict()
+ }))
return WriteResult(
method=WriteToBigQuery.Method.STORAGE_WRITE_API,
failed_rows=failed_rows,
failed_rows_with_errors=failed_rows_with_errors)
+ else:
+ raise ValueError(f"Unsupported method {method_to_use}")
+
def display_data(self):
res = {}
if self.table_reference is not None and isinstance(self.table_reference,
@@ -2487,7 +2509,7 @@ class StorageWriteToBigQuery(PTransform):
Experimental; no backwards compatibility guarantees.
"""
- URN = "beam:schematransform:org.apache.beam:bigquery_storage_write:v1"
+ URN = "beam:schematransform:org.apache.beam:bigquery_storage_write:v2"
FAILED_ROWS = "FailedRows"
FAILED_ROWS_WITH_ERRORS = "FailedRowsWithErrors"
@@ -2552,11 +2574,17 @@ def expand(self, input):
triggeringFrequencySeconds=self._triggering_frequency,
useAtLeastOnceSemantics=self._use_at_least_once,
writeDisposition=self._write_disposition,
- )
+ errorHandling={
+ 'output': StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS
+ })
input_tag = self.schematransform_config.inputs[0]
- return {input_tag: input} | external_storage_write
+ result = {input_tag: input} | external_storage_write
+ result[StorageWriteToBigQuery.FAILED_ROWS] = result[
+ StorageWriteToBigQuery.FAILED_ROWS_WITH_ERRORS] | beam.Map(
+ lambda row_and_error: row_and_error[0])
+ return result
class ReadFromBigQuery(PTransform):
@@ -2791,14 +2819,14 @@ def _expand_direct_read(self, pcoll):
else:
project_id = pcoll.pipeline.options.view_as(GoogleCloudOptions).project
+ pipeline_details = {}
+ if temp_table_ref is not None:
+ pipeline_details['temp_table_ref'] = temp_table_ref
+ elif project_id is not None:
+ pipeline_details['project_id'] = project_id
+ pipeline_details['bigquery_dataset_labels'] = self.bigquery_dataset_labels
+
def _get_pipeline_details(unused_elm):
- pipeline_details = {}
- if temp_table_ref is not None:
- pipeline_details['temp_table_ref'] = temp_table_ref
- elif project_id is not None:
- pipeline_details['project_id'] = project_id
- pipeline_details[
- 'bigquery_dataset_labels'] = self.bigquery_dataset_labels
return pipeline_details
project_to_cleanup_pcoll = beam.pvalue.AsList(
diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py
index e15205ead4d4..96f154dbe4b8 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -908,7 +908,8 @@ def expand(self, pbegin):
return (
pbegin
| Impulse()
- | core.Map(lambda _: self.source).with_output_types(BoundedSource)
+ | 'EmitSource' >>
+ core.Map(lambda _: self.source).with_output_types(BoundedSource)
| SDFBoundedSourceReader(display_data))
elif isinstance(self.source, ptransform.PTransform):
# The Read transform can also admit a full PTransform as an input
diff --git a/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py b/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py
index 3c3414923c83..8c902421f603 100644
--- a/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py
+++ b/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py
@@ -228,3 +228,6 @@ def run_inference(
return utils._convert_to_result(
batch, prediction.predictions, prediction.deployed_model_id)
+
+ def validate_inference_args(self, inference_args: Optional[Dict[str, Any]]):
+ pass
diff --git a/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py b/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py
index 02b4e5ec0703..4ef42fb10a70 100644
--- a/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py
+++ b/sdks/python/apache_beam/ml/inference/vertex_ai_inference_it_test.py
@@ -28,13 +28,15 @@
try:
from apache_beam.examples.inference import vertex_ai_image_classification
+ from apache_beam.examples.inference import vertex_ai_llm_text_classification
except ImportError as e:
raise unittest.SkipTest(
"Vertex AI model handler dependencies are not installed")
_INPUT = "gs://apache-beam-ml/testing/inputs/vertex_images/*/*.jpg"
_OUTPUT_DIR = "gs://apache-beam-ml/testing/outputs/vertex_images"
-_ENDPOINT_ID = "5384055553544683520"
+_FLOWER_ENDPOINT_ID = "5384055553544683520"
+_LLM_ENDPOINT_ID = "9157860935048626176"
_ENDPOINT_PROJECT = "apache-beam-testing"
_ENDPOINT_REGION = "us-central1"
_ENDPOINT_NETWORK = "projects/844138762903/global/networks/beam-test-vpc"
@@ -52,7 +54,7 @@ def test_vertex_ai_run_flower_image_classification(self):
extra_opts = {
'input': _INPUT,
'output': output_file,
- 'endpoint_id': _ENDPOINT_ID,
+ 'endpoint_id': _FLOWER_ENDPOINT_ID,
'endpoint_project': _ENDPOINT_PROJECT,
'endpoint_region': _ENDPOINT_REGION,
'endpoint_network': _ENDPOINT_NETWORK,
@@ -63,6 +65,22 @@ def test_vertex_ai_run_flower_image_classification(self):
test_pipeline.get_full_options_as_args(**extra_opts))
self.assertEqual(FileSystems().exists(output_file), True)
+ @pytest.mark.uses_vertex_ai
+ @pytest.mark.it_postcommit
+ def test_vertex_ai_run_llm_text_classification(self):
+ output_file = '/'.join([_OUTPUT_DIR, str(uuid.uuid4()), 'output.txt'])
+
+ test_pipeline = TestPipeline(is_integration_test=True)
+ extra_opts = {
+ 'output': output_file,
+ 'endpoint_id': _LLM_ENDPOINT_ID,
+ 'endpoint_project': _ENDPOINT_PROJECT,
+ 'endpoint_region': _ENDPOINT_REGION
+ }
+ vertex_ai_llm_text_classification.run(
+ test_pipeline.get_full_options_as_args(**extra_opts))
+ self.assertEqual(FileSystems().exists(output_file), True)
+
if __name__ == '__main__':
logging.getLogger().setLevel(logging.DEBUG)
diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index 6d9c5ecd37b8..3fbf7eff7dd6 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -101,7 +101,7 @@ def add_value_provider_argument(self, *args, **kwargs):
key/value form.
"""
# Extract the option name from positional argument ['pos_arg']
- assert args != () and len(args[0]) >= 1
+ assert args and len(args[0]) >= 1
if args[0][0] != '-':
option_name = args[0]
if kwargs.get('nargs') is None: # make them optionally templated
diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py
index e0944d81d1b2..18b422ed27d4 100644
--- a/sdks/python/apache_beam/pipeline.py
+++ b/sdks/python/apache_beam/pipeline.py
@@ -1330,7 +1330,10 @@ def transform_to_runner_api(
# Iterate over inputs and outputs by sorted key order, so that ids are
# consistently generated for multiple runs of the same pipeline.
- transform_spec = transform_to_runner_api(self.transform, context)
+ try:
+ transform_spec = transform_to_runner_api(self.transform, context)
+ except Exception as exn:
+ raise RuntimeError(f'Unable to translate {self.full_label}') from exn
environment_id = self.environment_id
transform_urn = transform_spec.urn if transform_spec else None
if (not environment_id and
diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py
index 2075c8eee3f1..8bb39940e484 100644
--- a/sdks/python/apache_beam/runners/dataflow/internal/names.py
+++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py
@@ -34,6 +34,6 @@
# Unreleased sdks use container image tag specified below.
# Update this tag whenever there is a change that
# requires changes to SDK harness container or SDK harness launcher.
-BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20230717'
+BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20230912'
DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3'
diff --git a/sdks/python/apache_beam/runners/render.py b/sdks/python/apache_beam/runners/render.py
index 55e15cb85629..306bf8c2090b 100644
--- a/sdks/python/apache_beam/runners/render.py
+++ b/sdks/python/apache_beam/runners/render.py
@@ -129,6 +129,12 @@ def _add_argparse_args(cls, parser):
help='Set to also log input pipeline proto to stdout.')
return parser
+ def __init__(self, *args, render_testing=False, **kwargs):
+ super().__init__(*args, **kwargs)
+ if self.render_port < 0 and not self.render_output and not render_testing:
+ raise ValueError(
+ 'At least one of --render_port or --render_output must be provided.')
+
class PipelineRenderer:
def __init__(self, pipeline, options):
@@ -147,8 +153,10 @@ def __init__(self, pipeline, options):
# Figure out at what point to stop rendering composite internals.
if options.render_leaf_composite_nodes:
- is_leaf = lambda name: any(
- re.match(pattern, name)
+ is_leaf = lambda transform_id: any(
+ re.match(
+ pattern,
+ self.pipeline.components.transforms[transform_id].unique_name)
for patterns in options.render_leaf_composite_nodes
for pattern in patterns.split(','))
self.leaf_composites = set()
@@ -403,6 +411,30 @@ def run_pipeline(self, pipeline_object, options, pipeline_proto=None):
if render_options.log_proto:
logging.info(pipeline_proto)
renderer = PipelineRenderer(pipeline_proto, render_options)
+ try:
+ subprocess.run(['dotX', '-V'], capture_output=True, check=True)
+ except FileNotFoundError as exn:
+ # If dot is not available, we can at least output the raw .dot files.
+ dot_files = [
+ output for output in render_options.render_output
+ if output.endswith('.dot')
+ ]
+ for output in dot_files:
+ with open(output, 'w') as fout:
+ fout.write(renderer.to_dot())
+ logging.info("Wrote pipeline as %s", output)
+
+ non_dot_files = set(render_options.render_output) - set(dot_files)
+ if non_dot_files:
+ raise RuntimeError(
+ "Graphviz dot executable not available "
+ f"for rendering non-dot output files {non_dot_files}") from exn
+ elif render_options.render_port >= 0:
+ raise RuntimeError(
+ "Graphviz dot executable not available for serving") from exn
+
+ return RenderPipelineResult(None)
+
renderer.page()
if render_options.render_port >= 0:
diff --git a/sdks/python/apache_beam/runners/render_test.py b/sdks/python/apache_beam/runners/render_test.py
index 5872e003aec7..4dca2b8b5221 100644
--- a/sdks/python/apache_beam/runners/render_test.py
+++ b/sdks/python/apache_beam/runners/render_test.py
@@ -83,6 +83,19 @@ def test_dot_well_formed(self):
renderer.update(toggle=[create_transform_id])
renderer.render_data()
+ def test_leaf_composite_filter(self):
+ try:
+ subprocess.run(['dot', '-V'], capture_output=True, check=True)
+ except FileNotFoundError:
+ self.skipTest('dot executable not installed')
+ p = beam.Pipeline()
+ _ = p | beam.Create([1, 2, 3]) | beam.Map(lambda x: x * x)
+ dot = render.PipelineRenderer(
+ p.to_runner_api(),
+ render.RenderOptions(['--render_leaf_composite_nodes=Create'],
+ render_testing=True)).to_dot()
+ self.assertEqual(dot.count('->'), 1)
+
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
diff --git a/sdks/python/apache_beam/runners/worker/data_sampler.py b/sdks/python/apache_beam/runners/worker/data_sampler.py
index a5992b9cebac..5ca307ca1b37 100644
--- a/sdks/python/apache_beam/runners/worker/data_sampler.py
+++ b/sdks/python/apache_beam/runners/worker/data_sampler.py
@@ -40,6 +40,8 @@
from apache_beam.coders.coder_impl import CoderImpl
from apache_beam.coders.coder_impl import WindowedValueCoderImpl
from apache_beam.coders.coders import Coder
+from apache_beam.options.pipeline_options import DebugOptions
+from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.portability.api import beam_fn_api_pb2
from apache_beam.utils.windowed_value import WindowedValue
@@ -49,11 +51,19 @@
class SampleTimer:
"""Periodic timer for sampling elements."""
def __init__(self, timeout_secs: float, sampler: OutputSampler) -> None:
- self._timeout_secs = timeout_secs
+ self._target_timeout_secs = timeout_secs
+ self._timeout_secs = min(timeout_secs, 0.5) if timeout_secs > 0 else 0.0
self._timer = Timer(self._timeout_secs, self.sample)
self._sampler = sampler
+ self._sample_duration_secs = 0.0
def reset(self) -> None:
+ # For the first 30 seconds, sample every 0.5 seconds. After that, sample at
+ # the normal rate.
+ if self._sample_duration_secs >= 30.0:
+ self._timeout_secs = self._target_timeout_secs
+ self._sample_duration_secs += self._timeout_secs
+
self._timer.cancel()
self._timer = Timer(self._timeout_secs, self.sample)
self._timer.start()
@@ -208,6 +218,7 @@ def __init__(
self,
max_samples: int = 10,
sample_every_sec: float = 30,
+ sample_only_exceptions: bool = False,
clock=None) -> None:
# Key is PCollection id. Is guarded by the _samplers_lock.
self._samplers: Dict[str, OutputSampler] = {}
@@ -215,10 +226,34 @@ def __init__(
# runner queries for samples.
self._samplers_lock: threading.Lock = threading.Lock()
self._max_samples = max_samples
- self._sample_every_sec = sample_every_sec
+ self._sample_every_sec = 0.0 if sample_only_exceptions else sample_every_sec
self._samplers_by_output: Dict[str, List[OutputSampler]] = {}
self._clock = clock
+ _ENABLE_DATA_SAMPLING = 'enable_data_sampling'
+ _ENABLE_ALWAYS_ON_EXCEPTION_SAMPLING = 'enable_always_on_exception_sampling'
+ _DISABLE_ALWAYS_ON_EXCEPTION_SAMPLING = 'disable_always_on_exception_sampling'
+
+ @staticmethod
+ def create(sdk_pipeline_options: PipelineOptions, **kwargs):
+ experiments = sdk_pipeline_options.view_as(DebugOptions).experiments or []
+
+ # When true, enables only the sampling of exceptions.
+ always_on_exception_sampling = (
+ DataSampler._ENABLE_ALWAYS_ON_EXCEPTION_SAMPLING in experiments and
+ DataSampler._DISABLE_ALWAYS_ON_EXCEPTION_SAMPLING not in experiments)
+
+ # When true, enables the sampling of all PCollections and exceptions.
+ enable_data_sampling = DataSampler._ENABLE_DATA_SAMPLING in experiments
+
+ if enable_data_sampling or always_on_exception_sampling:
+ sample_only_exceptions = (
+ always_on_exception_sampling and not enable_data_sampling)
+ return DataSampler(
+ sample_only_exceptions=sample_only_exceptions, **kwargs)
+ else:
+ return None
+
def stop(self) -> None:
"""Stops all sampling, does not clear samplers in case there are outstanding
samples.
diff --git a/sdks/python/apache_beam/runners/worker/data_sampler_test.py b/sdks/python/apache_beam/runners/worker/data_sampler_test.py
index 8d063fdb49d6..8c47315b7a9e 100644
--- a/sdks/python/apache_beam/runners/worker/data_sampler_test.py
+++ b/sdks/python/apache_beam/runners/worker/data_sampler_test.py
@@ -27,6 +27,7 @@
from apache_beam.coders import FastPrimitivesCoder
from apache_beam.coders import WindowedValueCoder
+from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.portability.api import beam_fn_api_pb2
from apache_beam.runners.worker.data_sampler import DataSampler
from apache_beam.runners.worker.data_sampler import OutputSampler
@@ -56,7 +57,9 @@ def make_test_descriptor(
return descriptor
def setUp(self):
- self.data_sampler = DataSampler(sample_every_sec=0.1)
+ self.data_sampler = DataSampler.create(
+ PipelineOptions(experiments=[DataSampler._ENABLE_DATA_SAMPLING]),
+ sample_every_sec=0.1)
def tearDown(self):
self.data_sampler.stop()
@@ -341,6 +344,103 @@ def test_can_sample_exceptions(self):
samples = self.data_sampler.wait_for_samples([MAIN_PCOLLECTION_ID])
self.assertGreater(len(samples.element_samples), 0)
+ def test_create_experiments(self):
+ """Tests that the experiments correctly make the DataSampler."""
+ enable_exception_exp = DataSampler._ENABLE_ALWAYS_ON_EXCEPTION_SAMPLING
+ disable_exception_exp = DataSampler._DISABLE_ALWAYS_ON_EXCEPTION_SAMPLING
+ enable_sampling_exp = DataSampler._ENABLE_DATA_SAMPLING
+
+ self.assertIsNone(DataSampler.create(PipelineOptions()))
+
+ exp = [disable_exception_exp]
+ self.assertIsNone(DataSampler.create(PipelineOptions(experiments=exp)))
+
+ exp = [enable_exception_exp, disable_exception_exp]
+ self.assertIsNone(DataSampler.create(PipelineOptions(experiments=exp)))
+
+ exp = [enable_exception_exp]
+ self.assertIsNotNone(DataSampler.create(PipelineOptions(experiments=exp)))
+
+ exp = [enable_sampling_exp]
+ self.assertIsNotNone(DataSampler.create(PipelineOptions(experiments=exp)))
+
+ exp = [enable_sampling_exp, enable_exception_exp, disable_exception_exp]
+ self.assertIsNotNone(DataSampler.create(PipelineOptions(experiments=exp)))
+
+ def test_samples_all_with_both_experiments(self):
+ """Tests that the using both sampling experiments samples everything."""
+ self.data_sampler = DataSampler.create(
+ PipelineOptions(
+ experiments=[
+ DataSampler._ENABLE_DATA_SAMPLING,
+ DataSampler._ENABLE_ALWAYS_ON_EXCEPTION_SAMPLING
+ ]),
+ sample_every_sec=0.1)
+
+ # Create a descriptor with one transform with two outputs, 'a' and 'b'.
+ descriptor = self.make_test_descriptor(outputs=['a', 'b'])
+ self.data_sampler.initialize_samplers(
+ MAIN_TRANSFORM_ID, descriptor, self.primitives_coder_factory)
+
+ # Get the samples for the two outputs.
+ a_sampler = self.data_sampler.sampler_for_output(MAIN_TRANSFORM_ID, 0)
+ b_sampler = self.data_sampler.sampler_for_output(MAIN_TRANSFORM_ID, 1)
+
+ # Sample an exception for the output 'a', this will show up in the final
+ # samples response.
+ exc_info = None
+ try:
+ raise Exception('test')
+ except Exception:
+ exc_info = sys.exc_info()
+ a_sampler.sample_exception('a', exc_info, MAIN_TRANSFORM_ID, 'instid')
+
+ # Sample a normal element for the output 'b', this will not show up in the
+ # final samples response.
+ b_sampler.element_sampler.el = 'b'
+ b_sampler.element_sampler.has_element = True
+
+ samples = self.data_sampler.wait_for_samples(['a', 'b'])
+ self.assertEqual(len(samples.element_samples), 2)
+ self.assertTrue(
+ samples.element_samples['a'].elements[0].HasField('exception'))
+ self.assertFalse(
+ samples.element_samples['b'].elements[0].HasField('exception'))
+
+ def test_only_sample_exceptions(self):
+ """Tests that the exception sampling experiment only samples exceptions."""
+ self.data_sampler = DataSampler.create(
+ PipelineOptions(
+ experiments=[DataSampler._ENABLE_ALWAYS_ON_EXCEPTION_SAMPLING]),
+ sample_every_sec=0.1)
+
+ # Create a descriptor with one transform with two outputs, 'a' and 'b'.
+ descriptor = self.make_test_descriptor(outputs=['a', 'b'])
+ self.data_sampler.initialize_samplers(
+ MAIN_TRANSFORM_ID, descriptor, self.primitives_coder_factory)
+
+ # Get the samples for the two outputs.
+ a_sampler = self.data_sampler.sampler_for_output(MAIN_TRANSFORM_ID, 0)
+ b_sampler = self.data_sampler.sampler_for_output(MAIN_TRANSFORM_ID, 1)
+
+ # Sample an exception for the output 'a', this will show up in the final
+ # samples response.
+ exc_info = None
+ try:
+ raise Exception('test')
+ except Exception:
+ exc_info = sys.exc_info()
+ a_sampler.sample_exception('a', exc_info, MAIN_TRANSFORM_ID, 'instid')
+
+ # Sample a normal element for the output 'b', this will not show up in the
+ # final samples response.
+ b_sampler.element_sampler.el = 'b'
+ b_sampler.element_sampler.has_element = True
+
+ samples = self.data_sampler.wait_for_samples([])
+ self.assertEqual(len(samples.element_samples), 1)
+ self.assertIsNotNone(samples.element_samples['a'].elements[0].exception)
+
class OutputSamplerTest(unittest.TestCase):
def tearDown(self):
diff --git a/sdks/python/apache_beam/runners/worker/operations.py b/sdks/python/apache_beam/runners/worker/operations.py
index 4b49afb14cd2..bdb16a46aeea 100644
--- a/sdks/python/apache_beam/runners/worker/operations.py
+++ b/sdks/python/apache_beam/runners/worker/operations.py
@@ -207,14 +207,16 @@ def update_counters_start(self, windowed_value):
# type: (WindowedValue) -> None
self.opcounter.update_from(windowed_value)
+ if self.execution_context is not None:
+ self.execution_context.output_sampler = self.output_sampler
+
# The following code is optimized by inlining a function call. Because this
# is called for every element, a function call is too expensive (order of
# 100s of nanoseconds). Furthermore, a lock was purposefully not used
# between here and the DataSampler as an additional operation. The tradeoff
# is that some samples might be dropped, but it is better than the
# alternative which is double sampling the same element.
- if self.element_sampler is not None and self.execution_context is not None:
- self.execution_context.output_sampler = self.output_sampler
+ if self.element_sampler is not None:
if not self.element_sampler.has_element:
self.element_sampler.el = windowed_value
self.element_sampler.has_element = True
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
index 87cf06e862ab..d3442fcb5987 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
@@ -153,9 +153,7 @@ def create_harness(environment, dry_run=False):
if dry_run:
return
- data_sampler = None
- if 'enable_data_sampling' in experiments:
- data_sampler = DataSampler()
+ data_sampler = DataSampler.create(sdk_pipeline_options)
sdk_harness = SdkHarness(
control_address=control_service_descriptor.url,
diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py
index 4ddf0b3e64a3..4b8e708bfc5c 100644
--- a/sdks/python/apache_beam/transforms/external.py
+++ b/sdks/python/apache_beam/transforms/external.py
@@ -194,6 +194,56 @@ def build(self):
return payload
+class ExplicitSchemaTransformPayloadBuilder(PayloadBuilder):
+ def __init__(self, identifier, schema_proto, **kwargs):
+ self._identifier = identifier
+ self._schema_proto = schema_proto
+ self._kwargs = kwargs
+
+ def build(self):
+ def dict_to_row_recursive(field_type, py_value):
+ if py_value is None:
+ return None
+ type_info = field_type.WhichOneof('type_info')
+ if type_info == 'row_type':
+ return dict_to_row(field_type.row_type.schema, py_value)
+ elif type_info == 'array_type':
+ return [
+ dict_to_row_recursive(field_type.array_type.element_type, value)
+ for value in py_value
+ ]
+ elif type_info == 'map_type':
+ return {
+ key: dict_to_row_recursive(field_type.map_type.value_type, value)
+ for key,
+ value in py_value.items()
+ }
+ else:
+ return py_value
+
+ def dict_to_row(schema_proto, py_value):
+ row_type = named_tuple_from_schema(schema_proto)
+ if isinstance(py_value, dict):
+ extra = set(py_value.keys()) - set(row_type._fields)
+ if extra:
+ raise ValueError(
+ f"Unknown fields: {extra}. Valid fields: {row_type._fields}")
+ return row_type(
+ *[
+ dict_to_row_recursive(
+ field.type, py_value.get(field.name, None))
+ for field in schema_proto.fields
+ ])
+ else:
+ return row_type(py_value)
+
+ return external_transforms_pb2.SchemaTransformPayload(
+ identifier=self._identifier,
+ configuration_schema=self._schema_proto,
+ configuration_row=RowCoder(self._schema_proto).encode(
+ dict_to_row(self._schema_proto, self._kwargs)))
+
+
class JavaClassLookupPayloadBuilder(PayloadBuilder):
"""
Builds a payload for directly instantiating a Java transform using a
@@ -351,37 +401,16 @@ def __init__(
_kwargs = kwargs
if rearrange_based_on_discovery:
- _kwargs = self._rearrange_kwargs(identifier)
-
- self._payload_builder = SchemaTransformPayloadBuilder(identifier, **_kwargs)
+ config = SchemaAwareExternalTransform.discover_config(
+ self._expansion_service, identifier)
+ self._payload_builder = ExplicitSchemaTransformPayloadBuilder(
+ identifier,
+ named_tuple_to_schema(config.configuration_schema),
+ **_kwargs)
- def _rearrange_kwargs(self, identifier):
- # discover and fetch the external SchemaTransform configuration then
- # use it to build an appropriate payload
- schematransform_config = SchemaAwareExternalTransform.discover_config(
- self._expansion_service, identifier)
-
- external_config_fields = schematransform_config.configuration_schema._fields
- ordered_kwargs = OrderedDict()
- missing_fields = []
-
- for field in external_config_fields:
- if field not in self._kwargs:
- missing_fields.append(field)
- else:
- ordered_kwargs[field] = self._kwargs[field]
-
- extra_fields = list(set(self._kwargs.keys()) - set(external_config_fields))
- if missing_fields:
- raise ValueError(
- 'Input parameters are missing the following SchemaTransform config '
- 'fields: %s' % missing_fields)
- elif extra_fields:
- raise ValueError(
- 'Input parameters include the following extra fields that are not '
- 'found in the SchemaTransform config schema: %s' % extra_fields)
-
- return ordered_kwargs
+ else:
+ self._payload_builder = SchemaTransformPayloadBuilder(
+ identifier, **_kwargs)
def expand(self, pcolls):
# Expand the transform using the expansion service.
@@ -390,14 +419,18 @@ def expand(self, pcolls):
self._payload_builder,
self._expansion_service)
- @staticmethod
- def discover(expansion_service):
+ @classmethod
+ @functools.lru_cache
+ def discover(cls, expansion_service, ignore_errors=False):
"""Discover all SchemaTransforms available to the given expansion service.
:return: a list of SchemaTransformsConfigs that represent the discovered
SchemaTransforms.
"""
+ return list(cls.discover_iter(expansion_service, ignore_errors))
+ @staticmethod
+ def discover_iter(expansion_service, ignore_errors=True):
with ExternalTransform.service(expansion_service) as service:
discover_response = service.DiscoverSchemaTransform(
beam_expansion_api_pb2.DiscoverSchemaTransformRequest())
@@ -406,8 +439,12 @@ def discover(expansion_service):
proto_config = discover_response.schema_transform_configs[identifier]
try:
schema = named_tuple_from_schema(proto_config.config_schema)
- except ValueError:
- continue
+ except Exception as exn:
+ if ignore_errors:
+ logging.info("Bad schema for %s: %s", identifier, str(exn)[:250])
+ continue
+ else:
+ raise
yield SchemaTransformsConfig(
identifier=identifier,
@@ -427,7 +464,8 @@ def discover_config(expansion_service, name):
are discovered
"""
- schematransforms = SchemaAwareExternalTransform.discover(expansion_service)
+ schematransforms = SchemaAwareExternalTransform.discover(
+ expansion_service, ignore_errors=True)
matched = []
for st in schematransforms:
diff --git a/sdks/python/apache_beam/transforms/external_test.py b/sdks/python/apache_beam/transforms/external_test.py
index fd5c81de6596..b0e39b7d9a5b 100644
--- a/sdks/python/apache_beam/transforms/external_test.py
+++ b/sdks/python/apache_beam/transforms/external_test.py
@@ -528,15 +528,19 @@ def test_rearrange_kwargs_based_on_discovery(self, mock_service):
kwargs = {"int_field": 0, "str_field": "str"}
transform = beam.SchemaAwareExternalTransform(
- identifier=identifier, expansion_service=expansion_service, **kwargs)
- ordered_kwargs = transform._rearrange_kwargs(identifier)
+ identifier=identifier,
+ expansion_service=expansion_service,
+ rearrange_based_on_discovery=True,
+ **kwargs)
+ payload = transform._payload_builder.build()
+ ordered_fields = [f.name for f in payload.configuration_schema.fields]
schematransform_config = beam.SchemaAwareExternalTransform.discover_config(
expansion_service, identifier)
external_config_fields = schematransform_config.configuration_schema._fields
self.assertNotEqual(tuple(kwargs.keys()), external_config_fields)
- self.assertEqual(tuple(ordered_kwargs.keys()), external_config_fields)
+ self.assertEqual(tuple(ordered_fields), external_config_fields)
class JavaClassLookupPayloadBuilderTest(unittest.TestCase):
diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py
index c7eaa152ae06..28614c6561c7 100644
--- a/sdks/python/apache_beam/transforms/ptransform.py
+++ b/sdks/python/apache_beam/transforms/ptransform.py
@@ -38,11 +38,13 @@ class and wrapper class that allows lambda functions to be used as
import copy
import itertools
+import json
import logging
import operator
import os
import sys
import threading
+import warnings
from functools import reduce
from functools import wraps
from typing import TYPE_CHECKING
@@ -83,6 +85,7 @@ class and wrapper class that allows lambda functions to be used as
from apache_beam.typehints.trivial_inference import instance_to_type
from apache_beam.typehints.typehints import validate_composite_type_param
from apache_beam.utils import proto_utils
+from apache_beam.utils import python_callable
if TYPE_CHECKING:
from apache_beam import coders
@@ -95,6 +98,7 @@ class and wrapper class that allows lambda functions to be used as
'PTransform',
'ptransform_fn',
'label_from_callable',
+ 'annotate_yaml',
]
_LOGGER = logging.getLogger(__name__)
@@ -1096,3 +1100,51 @@ def __ror__(self, pvalueish, _unused=None):
def expand(self, pvalue):
raise RuntimeError("Should never be expanded directly.")
+
+
+# Defined here to avoid circular import issues for Beam library transforms.
+def annotate_yaml(constructor):
+ """Causes instances of this transform to be annotated with their yaml syntax.
+
+ Should only be used for transforms that are fully defined by their constructor
+ arguments.
+ """
+ @wraps(constructor)
+ def wrapper(*args, **kwargs):
+ transform = constructor(*args, **kwargs)
+
+ fully_qualified_name = (
+ f'{constructor.__module__}.{constructor.__qualname__}')
+ try:
+ imported_constructor = (
+ python_callable.PythonCallableWithSource.
+ load_from_fully_qualified_name(fully_qualified_name))
+ if imported_constructor != wrapper:
+ raise ImportError('Different object.')
+ except ImportError:
+ warnings.warn(f'Cannot import {constructor} as {fully_qualified_name}.')
+ return transform
+
+ try:
+ config = json.dumps({
+ 'constructor': fully_qualified_name,
+ 'args': args,
+ 'kwargs': kwargs,
+ })
+ except TypeError as exn:
+ warnings.warn(
+ f'Cannot serialize arguments for {constructor} as json: {exn}')
+ return transform
+
+ original_annotations = transform.annotations
+ transform.annotations = lambda: {
+ **original_annotations(),
+ # These override whatever may have been provided earlier.
+ # The outermost call is expected to be the most specific.
+ 'yaml_provider': 'python',
+ 'yaml_type': 'PyTransform',
+ 'yaml_args': config,
+ }
+ return transform
+
+ return wrapper
diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py
index 4148e8627594..fb0e8e9789d8 100644
--- a/sdks/python/apache_beam/transforms/util.py
+++ b/sdks/python/apache_beam/transforms/util.py
@@ -1155,13 +1155,24 @@ def Iterables(delimiter=None):
class LogElements(PTransform):
"""
PTransform for printing the elements of a PCollection.
+
+ Args:
+ label (str): (optional) A custom label for the transform.
+ prefix (str): (optional) A prefix string to prepend to each logged element.
+ with_timestamp (bool): (optional) Whether to include element's timestamp.
+ with_window (bool): (optional) Whether to include element's window.
+ level: (optional) The logging level for the output (e.g. `logging.DEBUG`,
+ `logging.INFO`, `logging.WARNING`, `logging.ERROR`). If not specified,
+ the log is printed to stdout.
"""
class _LoggingFn(DoFn):
- def __init__(self, prefix='', with_timestamp=False, with_window=False):
+ def __init__(
+ self, prefix='', with_timestamp=False, with_window=False, level=None):
super().__init__()
self.prefix = prefix
self.with_timestamp = with_timestamp
self.with_window = with_window
+ self.level = level
def process(
self,
@@ -1178,19 +1189,38 @@ def process(
log_line += ', window(start=' + window.start.to_rfc3339()
log_line += ', end=' + window.end.to_rfc3339() + ')'
- print(log_line)
+ if self.level == logging.DEBUG:
+ logging.debug(log_line)
+ elif self.level == logging.INFO:
+ logging.info(log_line)
+ elif self.level == logging.WARNING:
+ logging.warning(log_line)
+ elif self.level == logging.ERROR:
+ logging.error(log_line)
+ elif self.level == logging.CRITICAL:
+ logging.critical(log_line)
+ else:
+ print(log_line)
+
yield element
def __init__(
- self, label=None, prefix='', with_timestamp=False, with_window=False):
+ self,
+ label=None,
+ prefix='',
+ with_timestamp=False,
+ with_window=False,
+ level=None):
super().__init__(label)
self.prefix = prefix
self.with_timestamp = with_timestamp
self.with_window = with_window
+ self.level = level
def expand(self, input):
return input | ParDo(
- self._LoggingFn(self.prefix, self.with_timestamp, self.with_window))
+ self._LoggingFn(
+ self.prefix, self.with_timestamp, self.with_window, self.level))
class Reify(object):
diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py
index 23aa5ecfc067..d8a8bacb96cd 100644
--- a/sdks/python/apache_beam/transforms/util_test.py
+++ b/sdks/python/apache_beam/transforms/util_test.py
@@ -1174,6 +1174,31 @@ def test_ptransform_output(self):
| util.LogElements(prefix='prefix_'))
assert_that(result, equal_to(['a', 'b', 'c']))
+ @pytest.fixture(scope="function")
+ def _capture_logs(request, caplog):
+ with caplog.at_level(logging.INFO):
+ with TestPipeline() as p:
+ _ = (
+ p | "info" >> beam.Create(["element"])
+ | "I" >> beam.LogElements(prefix='info_', level=logging.INFO))
+ _ = (
+ p | "warning" >> beam.Create(["element"])
+ | "W" >> beam.LogElements(prefix='warning_', level=logging.WARNING))
+ _ = (
+ p | "error" >> beam.Create(["element"])
+ | "E" >> beam.LogElements(prefix='error_', level=logging.ERROR))
+
+ request.captured_log = caplog.text
+
+ @pytest.mark.usefixtures("_capture_logs")
+ def test_setting_level_uses_appropriate_log_channel(self):
+ self.assertTrue(
+ re.compile('INFO(.*)info_element').search(self.captured_log))
+ self.assertTrue(
+ re.compile('WARNING(.*)warning_element').search(self.captured_log))
+ self.assertTrue(
+ re.compile('ERROR(.*)error_element').search(self.captured_log))
+
class ReifyTest(unittest.TestCase):
def test_timestamp(self):
diff --git a/sdks/python/apache_beam/typehints/schema_registry.py b/sdks/python/apache_beam/typehints/schema_registry.py
index 9ec7b1b65ccf..a73e97f43f70 100644
--- a/sdks/python/apache_beam/typehints/schema_registry.py
+++ b/sdks/python/apache_beam/typehints/schema_registry.py
@@ -40,13 +40,18 @@ def generate_new_id(self):
"schemas.")
def add(self, typing, schema):
- self.by_id[schema.id] = (typing, schema)
+ if not schema.id:
+ self.by_id[schema.id] = (typing, schema)
def get_typing_by_id(self, unique_id):
+ if not unique_id:
+ return None
result = self.by_id.get(unique_id, None)
return result[0] if result is not None else None
def get_schema_by_id(self, unique_id):
+ if not unique_id:
+ return None
result = self.by_id.get(unique_id, None)
return result[1] if result is not None else None
diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py
index 156b877d07e2..155ea86219de 100644
--- a/sdks/python/apache_beam/typehints/schemas.py
+++ b/sdks/python/apache_beam/typehints/schemas.py
@@ -87,6 +87,7 @@
from apache_beam.portability import common_urns
from apache_beam.portability.api import schema_pb2
from apache_beam.typehints import row_type
+from apache_beam.typehints import typehints
from apache_beam.typehints.native_type_compatibility import _get_args
from apache_beam.typehints.native_type_compatibility import _match_is_exactly_mapping
from apache_beam.typehints.native_type_compatibility import _match_is_optional
@@ -588,6 +589,25 @@ def named_fields_from_element_type(
return named_fields_from_schema(schema_from_element_type(element_type))
+def union_schema_type(element_types):
+ """Returns a schema whose fields are the union of each corresponding field.
+
+ element_types must be a set of schema-aware types whose fields have the
+ same naming and ordering.
+ """
+ union_fields_and_types = []
+ for field in zip(*[named_fields_from_element_type(t) for t in element_types]):
+ names, types = zip(*field)
+ name_set = set(names)
+ if len(name_set) != 1:
+ raise TypeError(
+ f"Could not determine schema for type hints {element_types!r}: "
+ f"Inconsistent names: {name_set}")
+ union_fields_and_types.append(
+ (next(iter(name_set)), typehints.Union[types]))
+ return named_tuple_from_schema(named_fields_to_schema(union_fields_and_types))
+
+
# Registry of typings for a schema by UUID
class LogicalTypeRegistry(object):
def __init__(self):
diff --git a/sdks/python/apache_beam/typehints/schemas_test.py b/sdks/python/apache_beam/typehints/schemas_test.py
index 2495f5722ad3..5d38b16d9783 100644
--- a/sdks/python/apache_beam/typehints/schemas_test.py
+++ b/sdks/python/apache_beam/typehints/schemas_test.py
@@ -22,6 +22,7 @@
import itertools
import pickle
import unittest
+from typing import Any
from typing import ByteString
from typing import List
from typing import Mapping
@@ -40,8 +41,10 @@
from apache_beam.portability import common_urns
from apache_beam.portability.api import schema_pb2
from apache_beam.typehints import row_type
+from apache_beam.typehints import typehints
from apache_beam.typehints.native_type_compatibility import match_is_named_tuple
from apache_beam.typehints.schemas import SchemaTypeRegistry
+from apache_beam.typehints.schemas import named_fields_from_element_type
from apache_beam.typehints.schemas import named_tuple_from_schema
from apache_beam.typehints.schemas import named_tuple_to_schema
from apache_beam.typehints.schemas import typing_from_runner_api
@@ -642,6 +645,16 @@ def test_row_type_is_callable(self):
self.assertIsInstance(instance, simple_row_type.user_type)
self.assertEqual(instance, (np.int64(35), 'baz'))
+ def test_union(self):
+ with_int = row_type.RowTypeConstraint.from_fields([('common', str),
+ ('unique', int)])
+ with_any = row_type.RowTypeConstraint.from_fields([('common', str),
+ ('unique', Any)])
+ union_type = typehints.Union[with_int, with_any]
+ self.assertEqual(
+ named_fields_from_element_type(union_type), [('common', str),
+ ('unique', Any)])
+
class HypothesisTest(unittest.TestCase):
# There is considerable variablility in runtime for this test, disable
diff --git a/sdks/python/apache_beam/typehints/trivial_inference.py b/sdks/python/apache_beam/typehints/trivial_inference.py
index f4b350e8f052..a880b5c70ea1 100644
--- a/sdks/python/apache_beam/typehints/trivial_inference.py
+++ b/sdks/python/apache_beam/typehints/trivial_inference.py
@@ -399,7 +399,10 @@ def infer_return_type_func(f, input_types, debug=False, depth=0):
jump_multiplier = 1
last_pc = -1
+ last_real_opname = opname = None
while pc < end: # pylint: disable=too-many-nested-blocks
+ if opname not in ('PRECALL', 'CACHE'):
+ last_real_opname = opname
start = pc
instruction = ofs_table[pc]
op = instruction.opcode
@@ -534,13 +537,13 @@ def infer_return_type_func(f, input_types, debug=False, depth=0):
return_type = Any
state.kw_names = None
else:
- # Handle lambdas always having an arg of 0 for CALL
+ # Handle comprehensions always having an arg of 0 for CALL
# See https://github.com/python/cpython/issues/102403 for context.
- if pop_count == 1:
- while pop_count <= len(state.stack):
- if isinstance(state.stack[-pop_count], Const):
- break
- pop_count += 1
+ if (pop_count == 1 and last_real_opname == 'GET_ITER' and
+ len(state.stack) > 1 and isinstance(state.stack[-2], Const) and
+ getattr(state.stack[-2].value, '__name__', None) in (
+ '', '', '', '')):
+ pop_count += 1
if depth <= 0 or pop_count > len(state.stack):
return_type = Any
elif isinstance(state.stack[-pop_count], Const):
diff --git a/sdks/python/apache_beam/typehints/trivial_inference_test.py b/sdks/python/apache_beam/typehints/trivial_inference_test.py
index d8cc2ab19a03..4341d11d3604 100644
--- a/sdks/python/apache_beam/typehints/trivial_inference_test.py
+++ b/sdks/python/apache_beam/typehints/trivial_inference_test.py
@@ -251,11 +251,30 @@ def testCall(self):
self.assertReturnType(
typehints.Tuple[int, typehints.Any], lambda: (1, f(x=1.0)))
+ def testCallNullaryMethod(self):
+ class Foo:
+ pass
+
+ self.assertReturnType(
+ typehints.Tuple[Foo, typehints.Any], lambda x: (x, x.unknown()), [Foo])
+
+ def testCallNestedLambda(self):
+ class Foo:
+ pass
+
+ self.assertReturnType(
+ typehints.Tuple[Foo, int], lambda x: (x, (lambda: 3)()), [Foo])
+
def testClosure(self):
x = 1
y = 1.0
self.assertReturnType(typehints.Tuple[int, float], lambda: (x, y))
+ @unittest.skip("https://github.com/apache/beam/issues/28420")
+ def testLocalClosure(self):
+ self.assertReturnType(
+ typehints.Tuple[int, int], lambda x: (x, (lambda: x)()), [int])
+
def testGlobals(self):
self.assertReturnType(int, lambda: global_int)
diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py
index ec6f4d10acf3..238bf8c321d6 100644
--- a/sdks/python/apache_beam/typehints/typehints.py
+++ b/sdks/python/apache_beam/typehints/typehints.py
@@ -604,6 +604,15 @@ def __getitem__(self, type_params):
return Any
elif len(params) == 1:
return next(iter(params))
+
+ if len(params) > 1:
+ from apache_beam.typehints import schemas
+ try:
+ return schemas.union_schema_type(params)
+ except (TypeError, KeyError):
+ # Not a union of compatible schema types.
+ pass
+
return self.UnionConstraint(params)
diff --git a/sdks/python/apache_beam/utils/python_callable.py b/sdks/python/apache_beam/utils/python_callable.py
index a7de214ec926..70aa7cb39e5c 100644
--- a/sdks/python/apache_beam/utils/python_callable.py
+++ b/sdks/python/apache_beam/utils/python_callable.py
@@ -77,7 +77,7 @@ def load_from_fully_qualified_name(fully_qualified_name):
return o
@staticmethod
- def load_from_script(source):
+ def load_from_script(source, method_name=None):
lines = [
line for line in source.split('\n')
if line.strip() and line.strip()[0] != '#'
@@ -85,26 +85,27 @@ def load_from_script(source):
common_indent = min(len(line) - len(line.lstrip()) for line in lines)
lines = [line[common_indent:] for line in lines]
- for ix, line in reversed(list(enumerate(lines))):
- if line[0] != ' ':
- if line.startswith('def '):
- name = line[4:line.index('(')].strip()
- elif line.startswith('class '):
- name = line[5:line.index('(') if '(' in
- line else line.index(':')].strip()
- else:
- name = '__python_callable__'
- lines[ix] = name + ' = ' + line
- break
- else:
- raise ValueError("Unable to identify callable from %r" % source)
+ if method_name is None:
+ for ix, line in reversed(list(enumerate(lines))):
+ if line[0] != ' ':
+ if line.startswith('def '):
+ method_name = line[4:line.index('(')].strip()
+ elif line.startswith('class '):
+ method_name = line[5:line.index('(') if '(' in
+ line else line.index(':')].strip()
+ else:
+ method_name = '__python_callable__'
+ lines[ix] = method_name + ' = ' + line
+ break
+ else:
+ raise ValueError("Unable to identify callable from %r" % source)
# pylint: disable=exec-used
# pylint: disable=ungrouped-imports
import apache_beam as beam
exec_globals = {'beam': beam}
exec('\n'.join(lines), exec_globals)
- return exec_globals[name]
+ return exec_globals[method_name]
def default_label(self):
src = self._source.strip()
diff --git a/sdks/python/apache_beam/utils/transform_service_launcher.py b/sdks/python/apache_beam/utils/transform_service_launcher.py
index 84f081e64ad9..33feab9bf29c 100644
--- a/sdks/python/apache_beam/utils/transform_service_launcher.py
+++ b/sdks/python/apache_beam/utils/transform_service_launcher.py
@@ -40,7 +40,7 @@
class TransformServiceLauncher(object):
_DEFAULT_PROJECT_NAME = 'apache.beam.transform.service'
- _DEFAULT_START_WAIT_TIMEOUT = 25000
+ _DEFAULT_START_WAIT_TIMEOUT = 50000
_launchers = {} # type: ignore
diff --git a/sdks/python/apache_beam/yaml/cache_provider_artifacts.py b/sdks/python/apache_beam/yaml/cache_provider_artifacts.py
new file mode 100644
index 000000000000..6c96dd3b0fd9
--- /dev/null
+++ b/sdks/python/apache_beam/yaml/cache_provider_artifacts.py
@@ -0,0 +1,46 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import logging
+import time
+
+from apache_beam.version import __version__ as beam_version
+from apache_beam.yaml import yaml_provider
+
+
+def cache_provider_artifacts():
+ providers_by_id = {}
+ for providers in yaml_provider.standard_providers().values():
+ for provider in providers:
+ # Dedup for better logging.
+ providers_by_id[id(provider)] = provider
+ for provider in providers_by_id.values():
+ t = time.time()
+ artifacts = provider.cache_artifacts()
+ if artifacts:
+ logging.info(
+ 'Cached %s in %0.03f seconds.', ', '.join(artifacts), time.time() - t)
+ if '.dev' not in beam_version:
+ # Also cache a base python venv for fast cloning.
+ t = time.time()
+ artifacts = yaml_provider.PypiExpansionService._create_venv_to_clone()
+ logging.info('Cached %s in %0.03f seconds.', artifacts, time.time() - t)
+
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ cache_provider_artifacts()
diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py
index 730d0f0ad0d1..eb0695f337b4 100644
--- a/sdks/python/apache_beam/yaml/main.py
+++ b/sdks/python/apache_beam/yaml/main.py
@@ -20,8 +20,13 @@
import yaml
import apache_beam as beam
+from apache_beam.typehints.schemas import LogicalType
+from apache_beam.typehints.schemas import MillisInstant
from apache_beam.yaml import yaml_transform
+# Workaround for https://github.com/apache/beam/issues/28151.
+LogicalType.register_logical_type(MillisInstant)
+
def _configure_parser(argv):
parser = argparse.ArgumentParser()
@@ -57,11 +62,14 @@ def run(argv=None):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pipeline_args,
pickle_library='cloudpickle',
- **pipeline_spec.get('options', {}))) as p:
+ **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get(
+ 'options', {})))) as p:
print("Building pipeline...")
yaml_transform.expand_pipeline(p, pipeline_spec)
print("Running pipeline...")
if __name__ == '__main__':
+ import logging
+ logging.getLogger().setLevel(logging.INFO)
run()
diff --git a/sdks/python/apache_beam/yaml/pipeline.schema.yaml b/sdks/python/apache_beam/yaml/pipeline.schema.yaml
index 575b38f16aed..ef0d9fe0f262 100644
--- a/sdks/python/apache_beam/yaml/pipeline.schema.yaml
+++ b/sdks/python/apache_beam/yaml/pipeline.schema.yaml
@@ -60,6 +60,10 @@ $defs:
type: array
items:
$ref: '#/$defs/transform'
+ extra_transforms:
+ type: array
+ items:
+ $ref: '#/$defs/transform'
source:
$ref: '#/$defs/transform'
sink:
@@ -100,7 +104,10 @@ $defs:
- if:
properties: { type: { const: composite }}
then:
- $ref: '#/$defs/nestedTransform'
+ allOf:
+ - $ref: '#/$defs/nestedTransform'
+ - properties:
+ extra_transforms: { not: {} }
- if:
properties: { type: { const: chain }}
then:
diff --git a/sdks/python/apache_beam/yaml/readme_test.py b/sdks/python/apache_beam/yaml/readme_test.py
index 7aac106228f2..958d9cb5783a 100644
--- a/sdks/python/apache_beam/yaml/readme_test.py
+++ b/sdks/python/apache_beam/yaml/readme_test.py
@@ -32,6 +32,7 @@
import apache_beam as beam
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.typehints import trivial_inference
+from apache_beam.yaml import yaml_provider
from apache_beam.yaml import yaml_transform
@@ -121,7 +122,7 @@ def expand(self, pcoll):
RENDER_DIR = None
-TEST_PROVIDERS = {
+TEST_TRANSFORMS = {
'Sql': FakeSql,
'ReadFromPubSub': FakeReadFromPubSub,
'WriteToPubSub': FakeWriteToPubSub,
@@ -129,6 +130,12 @@ def expand(self, pcoll):
}
+class TestProvider(yaml_provider.InlineProvider):
+ def _affinity(self, other):
+ # Always try to choose this one.
+ return float('inf')
+
+
class TestEnvironment:
def __enter__(self):
self.tempdir = tempfile.TemporaryDirectory()
@@ -196,8 +203,13 @@ def test(self):
os.path.join(RENDER_DIR, test_name + '.png')
]
options['render_leaf_composite_nodes'] = ['.*']
+ test_provider = TestProvider(TEST_TRANSFORMS)
p = beam.Pipeline(options=PipelineOptions(**options))
- yaml_transform.expand_pipeline(p, modified_yaml, TEST_PROVIDERS)
+ yaml_transform.expand_pipeline(
+ p,
+ modified_yaml,
+ {t: test_provider
+ for t in test_provider.provided_transforms()})
if test_type == 'BUILD':
return
p.run().wait_until_finish()
@@ -249,6 +261,10 @@ def createTestSuite(name, path):
ReadMeTest = createTestSuite(
'ReadMeTest', os.path.join(os.path.dirname(__file__), 'README.md'))
+ErrorHandlingTest = createTestSuite(
+ 'ErrorHandlingTest',
+ os.path.join(os.path.dirname(__file__), 'yaml_errors.md'))
+
if __name__ == '__main__':
parser = argparse.ArgumentParser()
parser.add_argument('--render_dir', default=None)
diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml
new file mode 100644
index 000000000000..e60f0026fd25
--- /dev/null
+++ b/sdks/python/apache_beam/yaml/standard_io.yaml
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+# This file enumerates the various IOs that are available by default as
+# top-level transforms in Beam's YAML.
+#
+# Note that there may be redundant implementations. In these cases the specs
+# should be kept in sync.
+# TODO(yaml): See if this can be enforced programmatically.
+
+- type: renaming
+ transforms:
+ 'ReadFromBigQuery': 'ReadFromBigQuery'
+ 'WriteToBigQuery': 'WriteToBigQuery'
+ config:
+ mappings:
+ 'ReadFromBigQuery':
+ query: 'query'
+ table: 'tableSpec'
+ fields: 'selectedFields'
+ row_restriction: 'rowRestriction'
+ 'WriteToBigQuery':
+ table: 'table'
+ create_disposition: 'createDisposition'
+ write_disposition: 'writeDisposition'
+ error_handling: 'errorHandling'
+ underlying_provider:
+ type: beamJar
+ transforms:
+ 'ReadFromBigQuery': 'beam:schematransform:org.apache.beam:bigquery_storage_read:v1'
+ 'WriteToBigQuery': 'beam:schematransform:org.apache.beam:bigquery_storage_write:v2'
+ config:
+ gradle_target: 'sdks:java:extensions:sql:expansion-service:shadowJar'
+
+- type: python
+ transforms:
+ 'ReadFromBigQuery': 'apache_beam.yaml.yaml_io.read_from_bigquery'
+ # Disable until https://github.com/apache/beam/issues/28162 is resolved.
+ # 'WriteToBigQuery': 'apache_beam.yaml.yaml_io.write_to_bigquery'
+
+# Declared as a renaming transform to avoid exposing all
+# (implementation-specific) pandas arguments and aligning with possible Java
+# implementation.
+# Invoking these directly as a PyTransform is still an option for anyone wanting
+# to use these power-features in a language-dependent manner.
+- type: renaming
+ transforms:
+ 'ReadFromCsv': 'ReadFromCsv'
+ 'WriteToCsv': 'WriteToCsv'
+ 'ReadFromJson': 'ReadFromJson'
+ 'WriteToJson': 'WriteToJson'
+ config:
+ mappings:
+ 'ReadFromCsv':
+ path: 'path'
+ 'WriteToCsv':
+ path: 'path'
+ 'ReadFromJson':
+ path: 'path'
+ 'WriteToJson':
+ path: 'path'
+ underlying_provider:
+ type: python
+ transforms:
+ 'ReadFromCsv': 'apache_beam.io.ReadFromCsv'
+ 'WriteToCsv': 'apache_beam.io.WriteToCsv'
+ 'ReadFromJson': 'apache_beam.io.ReadFromJson'
+ 'WriteToJson': 'apache_beam.io.WriteToJson'
diff --git a/sdks/python/apache_beam/yaml/yaml_errors.md b/sdks/python/apache_beam/yaml/yaml_errors.md
new file mode 100644
index 000000000000..e7a60f750a10
--- /dev/null
+++ b/sdks/python/apache_beam/yaml/yaml_errors.md
@@ -0,0 +1,196 @@
+
+
+# Beam YAML Error Handling
+
+The larger one's pipeline gets, the more common it is to encounter "exceptional"
+data that is malformatted, doesn't handle the proper preconditions, or otherwise
+breaks during processing. Generally any such record will cause the pipeline to
+permanently fail, but often it is desirable to allow the pipeline to continue,
+re-directing bad records to another path for special handling or simply
+recording them for later off-line analysis. This is often called the
+"dead letter queue" pattern.
+
+Beam YAML has special support for this pattern if the transform supports a
+`error_handling` config parameter with an `output` field. For example,
+the following code will write all "good" processed records to one file and
+any "bad" records to a separate file.
+
+```
+pipeline:
+ transforms:
+ - type: ReadFromCsv
+ config:
+ path: /path/to/input*.csv
+
+ - type: MapToFields
+ input: ReadFromCsv
+ config:
+ language: python
+ fields:
+ col1: col1
+ # This could raise a divide-by-zero error.
+ ratio: col2 / col3
+ error_handling:
+ output: my_error_output
+
+ - type: WriteToJson
+ input: MapToFields
+ config:
+ path: /path/to/output.json
+
+ - type: WriteToJson
+ name: WriteErrorsToJson
+ input: MapToFields.my_error_output
+ config:
+ path: /path/to/errors.json
+```
+
+Note that with `error_handling` declared, `MapToFields.my_error_output`
+**must** be consumed; to ignore it will be an error. Any use is fine, e.g.
+logging the bad records to stdout would be sufficient (though not recommended
+for a robust pipeline).
+
+Some transforms allow for extra arguments in their error_handling config, e.g.
+for Python functions one can give a `threshold` which limits the relative number
+of records that can be bad before considering the entire pipeline a failure
+
+```
+pipeline:
+ transforms:
+ - type: ReadFromCsv
+ config:
+ path: /path/to/input*.csv
+
+ - type: MapToFields
+ input: ReadFromCsv
+ config:
+ language: python
+ fields:
+ col1: col1
+ # This could raise a divide-by-zero error.
+ ratio: col2 / col3
+ error_handling:
+ output: my_error_output
+ # If more than 10% of records throw an error, stop the pipeline.
+ threshold: 0.1
+
+ - type: WriteToJson
+ input: MapToFields
+ config:
+ path: /path/to/output.json
+
+ - type: WriteToJson
+ name: WriteErrorsToJson
+ input: MapToFields.my_error_output
+ config:
+ path: /path/to/errors.json
+```
+
+One can do arbitrary further processing on these failed records if desired,
+e.g.
+
+```
+pipeline:
+ transforms:
+ - type: ReadFromCsv
+ config:
+ path: /path/to/input*.csv
+
+ - type: MapToFields
+ name: ComputeRatio
+ input: ReadFromCsv
+ config:
+ language: python
+ fields:
+ col1: col1
+ # This could raise a divide-by-zero error.
+ ratio: col2 / col3
+ error_handling:
+ output: my_error_output
+
+ - type: MapToFields
+ name: ComputeRatioForBadRecords
+ input: ComputeRatio.my_error_output
+ config:
+ language: python
+ fields:
+ col1: col1
+ ratio: col2 / (col3 + 1)
+ error_handling:
+ output: still_bad
+
+ - type: WriteToJson
+ # Takes as input everything from the "success" path of both transforms.
+ input: [ComputeRatio, ComputeRatioForBadRecords]
+ config:
+ path: /path/to/output.json
+
+ - type: WriteToJson
+ name: WriteErrorsToJson
+ # These failed the first and the second transform.
+ input: ComputeRatioForBadRecords.still_bad
+ config:
+ path: /path/to/errors.json
+```
+
+When using the `chain` syntax, the required error consumption can happen
+in an `extra_transforms` block.
+
+```
+pipeline:
+ type: chain
+ transforms:
+ - type: ReadFromCsv
+ config:
+ path: /path/to/input*.csv
+
+ - type: MapToFields
+ name: SomeStep
+ config:
+ language: python
+ fields:
+ col1: col1
+ # This could raise a divide-by-zero error.
+ ratio: col2 / col3
+ error_handling:
+ output: errors
+
+ - type: MapToFields
+ name: AnotherStep
+ config:
+ language: python
+ fields:
+ col1: col1
+ # This could raise a divide-by-zero error.
+ inverse_ratio: 1 / ratio
+ error_handling:
+ output: errors
+
+ - type: WriteToJson
+ config:
+ path: /path/to/output.json
+
+ extra_transforms:
+ - type: WriteToJson
+ name: WriteErrors
+ input: [SomeStep.errors, AnotherStep.errors]
+ config:
+ path: /path/to/errors.json
+```
diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py
new file mode 100644
index 000000000000..646d5e1fbff1
--- /dev/null
+++ b/sdks/python/apache_beam/yaml/yaml_io.py
@@ -0,0 +1,116 @@
+#
+# 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.
+#
+
+"""This module contains the Python implementations for the builtin IOs.
+
+They are referenced from standard_io.py.
+
+Note that in the case that they overlap with other (likely Java)
+implementations of the same transforms, the configs must be kept in sync.
+"""
+
+import os
+
+import yaml
+
+import apache_beam as beam
+from apache_beam.io import ReadFromBigQuery
+from apache_beam.io import WriteToBigQuery
+from apache_beam.io.gcp.bigquery import BigQueryDisposition
+from apache_beam.yaml import yaml_provider
+
+
+def read_from_bigquery(
+ query=None, table=None, row_restriction=None, fields=None):
+ if query is None:
+ assert table is not None
+ else:
+ assert table is None and row_restriction is None and fields is None
+ return ReadFromBigQuery(
+ query=query,
+ table=table,
+ row_restriction=row_restriction,
+ selected_fields=fields,
+ method='DIRECT_READ',
+ output_type='BEAM_ROW')
+
+
+def write_to_bigquery(
+ table,
+ *,
+ create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
+ write_disposition=BigQueryDisposition.WRITE_APPEND,
+ error_handling=None):
+ class WriteToBigQueryHandlingErrors(beam.PTransform):
+ def default_label(self):
+ return 'WriteToBigQuery'
+
+ def expand(self, pcoll):
+ write_result = pcoll | WriteToBigQuery(
+ table,
+ method=WriteToBigQuery.Method.STORAGE_WRITE_API
+ if error_handling else None,
+ create_disposition=create_disposition,
+ write_disposition=write_disposition,
+ temp_file_format='AVRO')
+ if error_handling and 'output' in error_handling:
+ # TODO: Support error rates.
+ return {
+ 'post_write': write_result.failed_rows_with_errors
+ | beam.FlatMap(lambda x: None),
+ error_handling['output']: write_result.failed_rows_with_errors
+ }
+ else:
+ if write_result._method == WriteToBigQuery.Method.FILE_LOADS:
+ # Never returns errors, just fails.
+ return {
+ 'post_write': write_result.destination_load_jobid_pairs
+ | beam.FlatMap(lambda x: None)
+ }
+ else:
+
+ # This should likely be pushed into the BQ read itself to avoid
+ # the possibility of silently ignoring errors.
+ def raise_exception(failed_row_with_error):
+ raise RuntimeError(failed_row_with_error.error_message)
+
+ _ = write_result.failed_rows_with_errors | beam.Map(raise_exception)
+ return {
+ 'post_write': write_result.failed_rows_with_errors
+ | beam.FlatMap(lambda x: None)
+ }
+
+ return WriteToBigQueryHandlingErrors()
+
+
+def io_providers():
+ with open(os.path.join(os.path.dirname(__file__), 'standard_io.yaml')) as fin:
+ explicit_ios = yaml_provider.parse_providers(
+ yaml.load(fin, Loader=yaml.SafeLoader))
+
+ # TOOD(yaml): We should make all top-level IOs explicit.
+ # This will be a chance to clean up the APIs and align them with their
+ # Java implementations.
+ # PythonTransform can be used to get the "raw" transforms for any others.
+ implicit_ios = yaml_provider.InlineProvider({
+ key: getattr(beam.io, key)
+ for key in dir(beam.io)
+ if (key.startswith('ReadFrom') or key.startswith('WriteTo')) and
+ key not in explicit_ios
+ })
+
+ return yaml_provider.merge_providers(explicit_ios, implicit_ios)
diff --git a/sdks/python/apache_beam/yaml/yaml_mapping.py b/sdks/python/apache_beam/yaml/yaml_mapping.py
index e845e514ecd1..b6dea894b3e9 100644
--- a/sdks/python/apache_beam/yaml/yaml_mapping.py
+++ b/sdks/python/apache_beam/yaml/yaml_mapping.py
@@ -16,10 +16,12 @@
#
"""This module defines the basic MapToFields operation."""
-
import itertools
+import js2py
+
import apache_beam as beam
+from apache_beam.io.filesystems import FileSystems
from apache_beam.typehints import row_type
from apache_beam.typehints import trivial_inference
from apache_beam.typehints.schemas import named_fields_from_element_type
@@ -27,33 +29,114 @@
from apache_beam.yaml import yaml_provider
-def _as_callable(original_fields, expr):
+def _check_mapping_arguments(
+ transform_name, expression=None, callable=None, name=None, path=None):
+ # Argument checking
+ if not expression and not callable and not path and not name:
+ raise ValueError(
+ f'{transform_name} must specify either "expression", "callable", '
+ f'or both "path" and "name"')
+ if expression and callable:
+ raise ValueError(
+ f'{transform_name} cannot specify both "expression" and "callable"')
+ if (expression or callable) and (path or name):
+ raise ValueError(
+ f'{transform_name} cannot specify "expression" or "callable" with '
+ f'"path" or "name"')
+ if path and not name:
+ raise ValueError(f'{transform_name} cannot specify "path" without "name"')
+ if name and not path:
+ raise ValueError(f'{transform_name} cannot specify "name" without "path"')
+
+
+# js2py's JsObjectWrapper object has a self-referencing __dict__ property
+# that cannot be pickled without implementing the __getstate__ and
+# __setstate__ methods.
+class _CustomJsObjectWrapper(js2py.base.JsObjectWrapper):
+ def __init__(self, js_obj):
+ super().__init__(js_obj.__dict__['_obj'])
+
+ def __getstate__(self):
+ return self.__dict__.copy()
+
+ def __setstate__(self, state):
+ self.__dict__.update(state)
+
+
+# TODO(yaml) Consider adding optional language version parameter to support
+# ECMAScript 5 and 6
+def _expand_javascript_mapping_func(
+ original_fields, expression=None, callable=None, path=None, name=None):
+ if expression:
+ args = ', '.join(original_fields)
+ js_func = f'function fn({args}) {{return ({expression})}}'
+ js_callable = _CustomJsObjectWrapper(js2py.eval_js(js_func))
+ return lambda __row__: js_callable(*__row__._asdict().values())
+
+ elif callable:
+ js_callable = _CustomJsObjectWrapper(js2py.eval_js(callable))
+ return lambda __row__: js_callable(__row__._asdict())
+
+ else:
+ if not path.endswith('.js'):
+ raise ValueError(f'File "{path}" is not a valid .js file.')
+ udf_code = FileSystems.open(path).read().decode()
+ js = js2py.EvalJs()
+ js.eval(udf_code)
+ js_callable = _CustomJsObjectWrapper(getattr(js, name))
+ return lambda __row__: js_callable(__row__._asdict())
+
+
+def _expand_python_mapping_func(
+ original_fields, expression=None, callable=None, path=None, name=None):
+ if path and name:
+ if not path.endswith('.py'):
+ raise ValueError(f'File "{path}" is not a valid .py file.')
+ py_file = FileSystems.open(path).read().decode()
+
+ return python_callable.PythonCallableWithSource.load_from_script(
+ py_file, name)
+
+ elif expression:
+ # TODO(robertwb): Consider constructing a single callable that takes
+ # the row and returns the new row, rather than invoking (and unpacking)
+ # for each field individually.
+ source = '\n'.join(['def fn(__row__):'] + [
+ f' {name} = __row__.{name}'
+ for name in original_fields if name in expression
+ ] + [' return (' + expression + ')'])
+
+ else:
+ source = callable
+
+ return python_callable.PythonCallableWithSource(source)
+
+
+def _as_callable(original_fields, expr, transform_name, language):
if expr in original_fields:
return expr
+
+ # TODO(yaml): support a type parameter
+ # TODO(yaml): support an imports parameter
+ # TODO(yaml): support a requirements parameter (possibly at a higher level)
+ if isinstance(expr, str):
+ expr = {'expression': expr}
+ if not isinstance(expr, dict):
+ raise ValueError(
+ f"Ambiguous expression type (perhaps missing quoting?): {expr}")
+ elif len(expr) != 1 and ('path' not in expr or 'name' not in expr):
+ raise ValueError(f"Ambiguous expression type: {list(expr.keys())}")
+
+ _check_mapping_arguments(transform_name, **expr)
+
+ if language == "javascript":
+ return _expand_javascript_mapping_func(original_fields, **expr)
+ elif language == "python":
+ return _expand_python_mapping_func(original_fields, **expr)
else:
- # TODO(yaml): support a type parameter
- # TODO(yaml): support an imports parameter
- # TODO(yaml): support a requirements parameter (possibly at a higher level)
- if isinstance(expr, str):
- expr = {'expression': expr}
- if not isinstance(expr, dict):
- raise ValueError(
- f"Ambiguous expression type (perhaps missing quoting?): {expr}")
- elif len(expr) != 1:
- raise ValueError(f"Ambiguous expression type: {list(expr.keys())}")
- if 'expression' in expr:
- # TODO(robertwb): Consider constructing a single callable that takes
- # the row and returns the new row, rather than invoking (and unpacking)
- # for each field individually.
- source = '\n'.join(['def fn(__row__):'] + [
- f' {name} = __row__.{name}'
- for name in original_fields if name in expr['expression']
- ] + [' return (' + expr['expression'] + ')'])
- elif 'callable' in expr:
- source = expr['callable']
- else:
- raise ValueError(f"Unknown expression type: {list(expr.keys())}")
- return python_callable.PythonCallableWithSource(source)
+ raise ValueError(
+ f'Unknown language for mapping transform: {language}. '
+ 'Supported languages are "javascript" and "python."')
# TODO(yaml): This should be available in all environments, in which case
@@ -88,14 +171,12 @@ def explode_zip(base, fields):
yield beam.Row(**copy)
return (
- beam.core._MaybePValueWithErrors(
- pcoll, self._exception_handling_args)
+ beam.core._MaybePValueWithErrors(pcoll, self._exception_handling_args)
| beam.FlatMap(
- lambda row: (
- explode_cross_product if self._cross_product else explode_zip)(
- {name: getattr(row, name) for name in all_fields}, # yapf
- to_explode))
- ).as_result()
+ lambda row:
+ (explode_cross_product if self._cross_product else explode_zip)
+ ({name: getattr(row, name)
+ for name in all_fields}, to_explode))).as_result()
def infer_output_type(self, input_type):
return row_type.RowTypeConstraint.from_fields([(
@@ -116,6 +197,8 @@ def _PythonProjectionTransform(
pcoll,
*,
fields,
+ transform_name,
+ language,
keep=None,
explode=(),
cross_product=True,
@@ -138,19 +221,16 @@ def _PythonProjectionTransform(
if isinstance(keep, str) and keep in original_fields:
keep_fn = lambda row: getattr(row, keep)
else:
- keep_fn = _as_callable(original_fields, keep)
+ keep_fn = _as_callable(original_fields, keep, transform_name, language)
filtered = pcoll | beam.Filter(keep_fn)
else:
filtered = pcoll
- if list(fields.items()) == [(name, name) for name in original_fields]:
- projected = filtered
- else:
- projected = filtered | beam.Select(
- **{
- name: _as_callable(original_fields, expr)
- for (name, expr) in fields.items()
- })
+ projected = filtered | beam.Select(
+ **{
+ name: _as_callable(original_fields, expr, transform_name, language)
+ for (name, expr) in fields.items()
+ })
if explode:
result = projected | _Explode(explode, cross_product=cross_product)
@@ -158,10 +238,10 @@ def _PythonProjectionTransform(
result = projected
return result.as_result(
- beam.MapTuple(
- lambda element,
- exc_info: beam.Row(
- element=element, msg=str(exc_info[1]), stack=str(exc_info[2]))))
+ # TODO(https://github.com/apache/beam/issues/24755): Switch to MapTuple.
+ beam.Map(
+ lambda x: beam.Row(
+ element=x[0], msg=str(x[1][1]), stack=str(x[1][2]))))
@beam.ptransform.ptransform_fn
@@ -177,8 +257,8 @@ def MapToFields(
drop=(),
language=None,
error_handling=None,
+ transform_name="MapToFields",
**language_keywords):
-
if isinstance(explode, str):
explode = [explode]
if cross_product is None:
@@ -242,13 +322,15 @@ def MapToFields(
return result
- elif language == 'python':
+ elif language == 'python' or language == 'javascript':
return pcoll | yaml_create_transform({
'type': 'PyTransform',
'config': {
'constructor': __name__ + '._PythonProjectionTransform',
'kwargs': {
'fields': fields,
+ 'transform_name': transform_name,
+ 'language': language,
'keep': keep,
'explode': explode,
'cross_product': cross_product,
@@ -281,6 +363,7 @@ def create_mapping_provider():
keep=keep,
fields={},
append=True,
+ transform_name='Filter',
**kwargs)),
'Explode': (
lambda yaml_create_transform,
@@ -290,5 +373,6 @@ def create_mapping_provider():
explode=explode,
fields={},
append=True,
+ transform_name='Explode',
**kwargs)),
})
diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py
index aa5aa7183318..6e035811d4b9 100644
--- a/sdks/python/apache_beam/yaml/yaml_provider.py
+++ b/sdks/python/apache_beam/yaml/yaml_provider.py
@@ -32,6 +32,7 @@
from typing import Dict
from typing import Iterable
from typing import Mapping
+from typing import Optional
import yaml
from yaml.loader import SafeLoader
@@ -57,6 +58,9 @@ def available(self) -> bool:
"""Returns whether this provider is available to use in this environment."""
raise NotImplementedError(type(self))
+ def cache_artifacts(self) -> Optional[Iterable[str]]:
+ raise NotImplementedError(type(self))
+
def provided_transforms(self) -> Iterable[str]:
"""Returns a list of transform type names this provider can handle."""
raise NotImplementedError(type(self))
@@ -72,6 +76,12 @@ def create_transform(
"""
raise NotImplementedError(type(self))
+ def underlying_provider(self):
+ """If this provider is simply a proxy to another provider, return the
+ provider that should actually be used for affinity checking.
+ """
+ return self
+
def affinity(self, other: "Provider"):
"""Returns a value approximating how good it would be for this provider
to be used immediately following a transform from the other provider
@@ -81,7 +91,9 @@ def affinity(self, other: "Provider"):
# E.g. we could look at the the expected environments themselves.
# Possibly, we could provide multiple expansions and have the runner itself
# choose the actual implementation based on fusion (and other) criteria.
- return self._affinity(other) + other._affinity(self)
+ return (
+ self.underlying_provider()._affinity(other) +
+ other.underlying_provider()._affinity(self))
def _affinity(self, other: "Provider"):
if self is other or self == other:
@@ -122,16 +134,18 @@ def create_transform(self, type, args, yaml_create_transform):
self._service = self._service()
if self._schema_transforms is None:
try:
- self._schema_transforms = [
- config.identifier
+ self._schema_transforms = {
+ config.identifier: config
for config in external.SchemaAwareExternalTransform.discover(
- self._service)
- ]
+ self._service, ignore_errors=True)
+ }
except Exception:
- self._schema_transforms = []
+ # It's possible this service doesn't vend schema transforms.
+ self._schema_transforms = {}
urn = self._urns[type]
if urn in self._schema_transforms:
- return external.SchemaAwareExternalTransform(urn, self._service, **args)
+ return external.SchemaAwareExternalTransform(
+ urn, self._service, rearrange_based_on_discovery=True, **args)
else:
return type >> self.create_external_transform(urn, args)
@@ -143,10 +157,21 @@ def create_external_transform(self, urn, args):
@classmethod
def provider_from_spec(cls, spec):
+ from apache_beam.yaml.yaml_transform import SafeLineLoader
+ for required in ('type', 'transforms'):
+ if required not in spec:
+ raise ValueError(
+ f'Missing {required} in provider '
+ f'at line {SafeLineLoader.get_line(spec)}')
urns = spec['transforms']
type = spec['type']
- from apache_beam.yaml.yaml_transform import SafeLineLoader
config = SafeLineLoader.strip_metadata(spec.get('config', {}))
+ extra_params = set(SafeLineLoader.strip_metadata(spec).keys()) - set(
+ ['transforms', 'type', 'config'])
+ if extra_params:
+ raise ValueError(
+ f'Unexpected parameters in provider of type {type} '
+ f'at line {SafeLineLoader.get_line(spec)}: {extra_params}')
if config.get('version', None) == 'BEAM_VERSION':
config['version'] = beam_version
if type in cls._provider_types:
@@ -235,17 +260,37 @@ def available(self):
self._is_available = False
return self._is_available
+ def cache_artifacts(self):
+ pass
+
class ExternalJavaProvider(ExternalProvider):
def __init__(self, urns, jar_provider):
super().__init__(
urns, lambda: external.JavaJarExpansionService(jar_provider()))
+ self._jar_provider = jar_provider
def available(self):
# pylint: disable=subprocess-run-check
return subprocess.run(['which', 'java'],
capture_output=True).returncode == 0
+ def cache_artifacts(self):
+ return [self._jar_provider()]
+
+
+@ExternalProvider.register_provider_type('python')
+def python(urns, packages=()):
+ if packages:
+ return ExternalPythonProvider(urns, packages)
+ else:
+ return InlineProvider({
+ name:
+ python_callable.PythonCallableWithSource.load_from_fully_qualified_name(
+ constructor)
+ for (name, constructor) in urns.items()
+ })
+
@ExternalProvider.register_provider_type('pythonPackage')
class ExternalPythonProvider(ExternalProvider):
@@ -255,6 +300,9 @@ def __init__(self, urns, packages):
def available(self):
return True # If we're running this script, we have Python installed.
+ def cache_artifacts(self):
+ return [self._service._venv()]
+
def create_external_transform(self, urn, args):
# Python transforms are "registered" by fully qualified name.
return external.ExternalTransform(
@@ -317,6 +365,9 @@ def __init__(self, transform_factories):
def available(self):
return True
+ def cache_artifacts(self):
+ pass
+
def provided_transforms(self):
return self._transform_factories.keys()
@@ -339,7 +390,35 @@ def create_transform(self, type, args, yaml_create_transform):
}
+def dicts_to_rows(o):
+ if isinstance(o, dict):
+ return beam.Row(**{k: dicts_to_rows(v) for k, v in o.items()})
+ elif isinstance(o, list):
+ return [dicts_to_rows(e) for e in o]
+ else:
+ return o
+
+
def create_builtin_provider():
+ def create(elements: Iterable[Any], reshuffle: bool = True):
+ """Creates a collection containing a specified set of elements.
+
+ YAML/JSON-style mappings will be interpreted as Beam rows. For example::
+
+ type: Create
+ elements:
+ - {first: 0, second: {str: "foo", values: [1, 2, 3]}}
+
+ will result in a schema of the form (int, Row(string, List[int])).
+
+ Args:
+ elements: The set of elements that should belong to the PCollection.
+ YAML/JSON-style mappings will be interpreted as Beam rows.
+ reshuffle (optional): Whether to introduce a reshuffle if there is more
+ than one element in the collection. Defaults to True.
+ """
+ return beam.Create(dicts_to_rows(elements), reshuffle)
+
def with_schema(**args):
# TODO: This is preliminary.
def parse_type(spec):
@@ -429,16 +508,9 @@ def _parse_window_spec(spec):
# TODO: Triggering, etc.
return beam.WindowInto(window_fn)
- ios = {
- key: getattr(apache_beam.io, key)
- for key in dir(apache_beam.io)
- if key.startswith('ReadFrom') or key.startswith('WriteTo')
- }
-
return InlineProvider(
dict({
- 'Create': lambda elements,
- reshuffle=True: beam.Create(elements, reshuffle),
+ 'Create': create,
'PyMap': lambda fn: beam.Map(
python_callable.PythonCallableWithSource(fn)),
'PyMapTuple': lambda fn: beam.MapTuple(
@@ -459,8 +531,7 @@ def _parse_window_spec(spec):
'Flatten': Flatten,
'WindowInto': WindowInto,
'GroupByKey': beam.GroupByKey,
- },
- **ios))
+ }))
class PypiExpansionService:
@@ -473,23 +544,60 @@ def __init__(self, packages, base_python=sys.executable):
self._packages = packages
self._base_python = base_python
- def _key(self):
- return json.dumps({'binary': self._base_python, 'packages': self._packages})
+ @classmethod
+ def _key(cls, base_python, packages):
+ return json.dumps({
+ 'binary': base_python, 'packages': sorted(packages)
+ },
+ sort_keys=True)
- def _venv(self):
- venv = os.path.join(
- self.VENV_CACHE,
- hashlib.sha256(self._key().encode('utf-8')).hexdigest())
+ @classmethod
+ def _path(cls, base_python, packages):
+ return os.path.join(
+ cls.VENV_CACHE,
+ hashlib.sha256(cls._key(base_python,
+ packages).encode('utf-8')).hexdigest())
+
+ @classmethod
+ def _create_venv_from_scratch(cls, base_python, packages):
+ venv = cls._path(base_python, packages)
if not os.path.exists(venv):
- python_binary = os.path.join(venv, 'bin', 'python')
- subprocess.run([self._base_python, '-m', 'venv', venv], check=True)
- subprocess.run([python_binary, '-m', 'ensurepip'], check=True)
- subprocess.run([python_binary, '-m', 'pip', 'install'] + self._packages,
+ subprocess.run([base_python, '-m', 'venv', venv], check=True)
+ venv_python = os.path.join(venv, 'bin', 'python')
+ subprocess.run([venv_python, '-m', 'ensurepip'], check=True)
+ subprocess.run([venv_python, '-m', 'pip', 'install'] + packages,
check=True)
with open(venv + '-requirements.txt', 'w') as fout:
- fout.write('\n'.join(self._packages))
+ fout.write('\n'.join(packages))
return venv
+ @classmethod
+ def _create_venv_from_clone(cls, base_python, packages):
+ venv = cls._path(base_python, packages)
+ if not os.path.exists(venv):
+ 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)
+ venv_binary = os.path.join(venv, 'bin', 'python')
+ subprocess.run([venv_binary, '-m', 'pip', 'install'] + packages,
+ check=True)
+ with open(venv + '-requirements.txt', 'w') as fout:
+ fout.write('\n'.join(packages))
+ return venv
+
+ @classmethod
+ def _create_venv_to_clone(cls, base_python):
+ return cls._create_venv_from_scratch(
+ base_python, [
+ 'apache_beam[dataframe,gcp,test]==' + beam_version,
+ 'virtualenv-clone'
+ ])
+
+ def _venv(self):
+ return self._create_venv_from_clone(self._base_python, self._packages)
+
def __enter__(self):
venv = self._venv()
self._service_provider = subprocess_server.SubprocessServer(
@@ -512,6 +620,50 @@ def __exit__(self, *args):
self._service = None
+@ExternalProvider.register_provider_type('renaming')
+class RenamingProvider(Provider):
+ def __init__(self, transforms, mappings, underlying_provider):
+ if isinstance(underlying_provider, dict):
+ underlying_provider = ExternalProvider.provider_from_spec(
+ underlying_provider)
+ self._transforms = transforms
+ self._underlying_provider = underlying_provider
+ for transform in transforms.keys():
+ if transform not in mappings:
+ raise ValueError(f'Missing transform {transform} in mappings.')
+ self._mappings = mappings
+
+ def available(self) -> bool:
+ return self._underlying_provider.available()
+
+ def provided_transforms(self) -> Iterable[str]:
+ return self._transforms.keys()
+
+ def create_transform(
+ self,
+ typ: str,
+ args: Mapping[str, Any],
+ yaml_create_transform: Callable[
+ [Mapping[str, Any], Iterable[beam.PCollection]], beam.PTransform]
+ ) -> beam.PTransform:
+ """Creates a PTransform instance for the given transform type and arguments.
+ """
+ mappings = self._mappings[typ]
+ remapped_args = {
+ mappings.get(key, key): value
+ for key, value in args.items()
+ }
+ return self._underlying_provider.create_transform(
+ self._transforms[typ], remapped_args, yaml_create_transform)
+
+ def _affinity(self, other):
+ raise NotImplementedError(
+ 'Should not be calling _affinity directly on this provider.')
+
+ def underlying_provider(self):
+ return self._underlying_provider.underlying_provider()
+
+
def parse_providers(provider_specs):
providers = collections.defaultdict(list)
for provider_spec in provider_specs:
@@ -539,10 +691,12 @@ def merge_providers(*provider_sets):
def standard_providers():
from apache_beam.yaml.yaml_mapping import create_mapping_provider
+ from apache_beam.yaml.yaml_io import io_providers
with open(os.path.join(os.path.dirname(__file__),
'standard_providers.yaml')) as fin:
standard_providers = yaml.load(fin, Loader=SafeLoader)
return merge_providers(
create_builtin_provider(),
create_mapping_provider(),
+ io_providers(),
parse_providers(standard_providers))
diff --git a/sdks/python/apache_beam/yaml/yaml_transform.py b/sdks/python/apache_beam/yaml/yaml_transform.py
index 70cbf0b7cee3..8bee2ccf2b98 100644
--- a/sdks/python/apache_beam/yaml/yaml_transform.py
+++ b/sdks/python/apache_beam/yaml/yaml_transform.py
@@ -22,8 +22,11 @@
import pprint
import re
import uuid
+from typing import Any
from typing import Iterable
+from typing import List
from typing import Mapping
+from typing import Set
import yaml
from yaml.loader import SafeLoader
@@ -162,13 +165,31 @@ def get_transform_id(self, transform_name):
class Scope(LightweightScope):
"""To look up PCollections (typically outputs of prior transforms) by name."""
- def __init__(self, root, inputs, transforms, providers, input_providers):
+ def __init__(
+ self,
+ root,
+ inputs: Mapping[str, Any],
+ transforms: Iterable[dict],
+ providers: Mapping[str, Iterable[yaml_provider.Provider]],
+ input_providers: Iterable[yaml_provider.Provider]):
super().__init__(transforms)
self.root = root
self._inputs = inputs
self.providers = providers
- self._seen_names = set()
+ self._seen_names: Set[str] = set()
self.input_providers = input_providers
+ self._all_followers = None
+
+ def followers(self, transform_name):
+ if self._all_followers is None:
+ self._all_followers = collections.defaultdict(list)
+ # TODO(yaml): Also trace through outputs and composites.
+ for transform in self._transforms:
+ if transform['type'] != 'composite':
+ for input in transform.get('input').values():
+ transform_id, _ = self.get_transform_id_and_output_name(input)
+ self._all_followers[transform_id].append(transform['__uuid__'])
+ return self._all_followers[self.get_transform_id(transform_name)]
def compute_all(self):
for transform_id in self._transforms_by_uuid.keys():
@@ -208,6 +229,77 @@ def get_outputs(self, transform_name):
def compute_outputs(self, transform_id):
return expand_transform(self._transforms_by_uuid[transform_id], self)
+ def best_provider(
+ self, t, input_providers: yaml_provider.Iterable[yaml_provider.Provider]):
+ if isinstance(t, dict):
+ spec = t
+ else:
+ spec = self._transforms_by_uuid[self.get_transform_id(t)]
+ possible_providers = [
+ p for p in self.providers[spec['type']] if p.available()
+ ]
+ if not possible_providers:
+ raise ValueError(
+ 'No available provider for type %r at %s' %
+ (spec['type'], identify_object(spec)))
+ # From here on, we have the invariant that possible_providers is not empty.
+
+ # Only one possible provider, no need to rank further.
+ if len(possible_providers) == 1:
+ return possible_providers[0]
+
+ def best_matches(
+ possible_providers: Iterable[yaml_provider.Provider],
+ adjacent_provider_options: Iterable[Iterable[yaml_provider.Provider]]
+ ) -> List[yaml_provider.Provider]:
+ """Given a set of possible providers, and a set of providers for each
+ adjacent transform, returns the top possible providers as ranked by
+ affinity to the adjacent transforms' providers.
+ """
+ providers_by_score = collections.defaultdict(list)
+ for p in possible_providers:
+ # The sum of the affinity of the best provider
+ # for each adjacent transform.
+ providers_by_score[sum(
+ max(p.affinity(ap) for ap in apo)
+ for apo in adjacent_provider_options)].append(p)
+ return providers_by_score[max(providers_by_score.keys())]
+
+ # If there are any inputs, prefer to match them.
+ if input_providers:
+ possible_providers = best_matches(
+ possible_providers, [[p] for p in input_providers])
+
+ # Without __uuid__ we can't find downstream operations.
+ if '__uuid__' not in spec:
+ return possible_providers[0]
+
+ # Match against downstream transforms, continuing until there is no tie
+ # or we run out of downstream transforms.
+ if len(possible_providers) > 1:
+ adjacent_transforms = list(self.followers(spec['__uuid__']))
+ while adjacent_transforms:
+ # This is a list of all possible providers for each adjacent transform.
+ adjacent_provider_options = [[
+ p for p in self.providers[self._transforms_by_uuid[t]['type']]
+ if p.available()
+ ] for t in adjacent_transforms]
+ if any(not apo for apo in adjacent_provider_options):
+ # One of the transforms had no available providers.
+ # We will throw an error later, doesn't matter what we return.
+ break
+ # Filter down the set of possible providers to the best ones.
+ possible_providers = best_matches(
+ possible_providers, adjacent_provider_options)
+ # If we are down to one option, no need to go further.
+ if len(possible_providers) == 1:
+ break
+ # Go downstream one more step.
+ adjacent_transforms = sum(
+ [list(self.followers(t)) for t in adjacent_transforms], [])
+
+ return possible_providers[0]
+
# A method on scope as providers may be scoped...
def create_ptransform(self, spec, input_pcolls):
if 'type' not in spec:
@@ -225,19 +317,7 @@ def create_ptransform(self, spec, input_pcolls):
providers_by_input[pcoll] for pcoll in input_pcolls
if pcoll in providers_by_input
]
-
- def provider_score(p):
- return sum(p.affinity(o) for o in input_providers)
-
- for provider in sorted(self.providers.get(spec['type']),
- key=provider_score,
- reverse=True):
- if provider.available():
- break
- else:
- raise ValueError(
- 'No available provider for type %r at %s' %
- (spec['type'], identify_object(spec)))
+ provider = self.best_provider(spec, input_providers)
config = SafeLineLoader.strip_metadata(spec.get('config', {}))
if not isinstance(config, dict):
@@ -293,8 +373,11 @@ def unique_name(self, spec, ptransform, strictness=0):
if 'name' in spec:
name = spec['name']
strictness += 1
- else:
+ elif 'ExternalTransform' not in ptransform.label:
+ # The label may have interesting information.
name = ptransform.label
+ else:
+ name = spec['type']
if name in self._seen_names:
if strictness >= 2:
raise ValueError(f'Duplicate name at {identify_object(spec)}: {name}')
@@ -422,6 +505,7 @@ def is_not_output_of_last_transform(new_transforms, value):
else:
transform['input'] = new_transforms[-1]['__uuid__']
new_transforms.append(transform)
+ new_transforms.extend(spec.get('extra_transforms', []))
composite_spec['transforms'] = new_transforms
last_transform = new_transforms[-1]['__uuid__']
@@ -506,14 +590,19 @@ def identify_object(spec):
def extract_name(spec):
- if 'name' in spec:
- return spec['name']
- elif 'id' in spec:
- return spec['id']
- elif 'type' in spec:
- return spec['type']
- elif len(spec) == 1:
- return extract_name(next(iter(spec.values())))
+ if isinstance(spec, dict):
+ if 'name' in spec:
+ return spec['name']
+ elif 'id' in spec:
+ return spec['id']
+ elif 'type' in spec:
+ return spec['type']
+ elif len(spec) == 1:
+ return extract_name(next(iter(spec.values())))
+ else:
+ return ''
+ elif isinstance(spec, str):
+ return spec
else:
return ''
@@ -598,7 +687,9 @@ def preprocess_windowing(spec):
'type': 'WindowInto',
'name': f'WindowInto[{out}]',
'windowing': windowing,
- 'input': modified_spec['__uuid__'] + ('.' + out if out else ''),
+ 'input': {
+ 'input': modified_spec['__uuid__'] + ('.' + out if out else '')
+ },
'__line__': spec['__line__'],
'__uuid__': SafeLineLoader.create_uuid(),
} for out in consumed_outputs]
@@ -718,7 +809,7 @@ def ensure_config(spec):
return spec
-def preprocess(spec, verbose=False):
+def preprocess(spec, verbose=False, known_transforms=None):
if verbose:
pprint.pprint(spec)
@@ -729,8 +820,19 @@ def apply(phase, spec):
spec, transforms=[apply(phase, t) for t in spec['transforms']])
return spec
+ if known_transforms:
+ known_transforms = set(known_transforms).union(['chain', 'composite'])
+
+ def ensure_transforms_have_providers(spec):
+ if known_transforms:
+ if spec['type'] not in known_transforms:
+ raise ValueError(
+ f'Unknown type or missing provider for {identify_object(spec)}')
+ return spec
+
for phase in [
ensure_transforms_have_types,
+ ensure_transforms_have_providers,
preprocess_source_sink,
preprocess_chain,
normalize_inputs_outputs,
@@ -753,13 +855,13 @@ def __init__(self, spec, providers={}): # pylint: disable=dangerous-default-val
if isinstance(spec, str):
spec = yaml.load(spec, Loader=SafeLineLoader)
# TODO(BEAM-26941): Validate as a transform.
- self._spec = preprocess(spec)
self._providers = yaml_provider.merge_providers(
{
key: yaml_provider.as_provider_list(key, value)
for (key, value) in providers.items()
},
yaml_provider.standard_providers())
+ self._spec = preprocess(spec, known_transforms=self._providers.keys())
def expand(self, pcolls):
if isinstance(pcolls, beam.pvalue.PBegin):
diff --git a/sdks/python/apache_beam/yaml/yaml_transform_scope_test.py b/sdks/python/apache_beam/yaml/yaml_transform_scope_test.py
index a22e4f851a1f..733f47583a7f 100644
--- a/sdks/python/apache_beam/yaml/yaml_transform_scope_test.py
+++ b/sdks/python/apache_beam/yaml/yaml_transform_scope_test.py
@@ -15,6 +15,7 @@
# limitations under the License.
#
+import collections
import logging
import unittest
@@ -22,6 +23,7 @@
import apache_beam as beam
from apache_beam.yaml import yaml_provider
+from apache_beam.yaml import yaml_transform
from apache_beam.yaml.yaml_transform import LightweightScope
from apache_beam.yaml.yaml_transform import SafeLineLoader
from apache_beam.yaml.yaml_transform import Scope
@@ -136,6 +138,146 @@ def test_create_ptransform_with_inputs(self):
self.assertDictEqual(result_annotations, target_annotations)
+class TestProvider(yaml_provider.InlineProvider):
+ def __init__(self, transform, name):
+ super().__init__({
+ name: lambda: beam.Map(lambda x: (x or ()) + (name, )), # or None
+ transform: lambda: beam.Map(lambda x: (x or ()) + (name, )),
+ })
+ self._transform = transform
+ self._name = name
+
+ def __repr__(self):
+ return 'TestProvider(%r, %r)' % (self._transform, self._name)
+
+ def _affinity(self, other):
+ if isinstance(other, TestProvider):
+ # Providers are closer based on how much their names match prefixes.
+ affinity = 1
+ for x, y in zip(self._name, other._name):
+ if x != y:
+ break
+ affinity *= 10
+ return affinity
+ else:
+ return -1000
+
+
+class ProviderAffinityTest(unittest.TestCase):
+ @staticmethod
+ def create_scope(s, providers):
+ providers_dict = collections.defaultdict(list)
+ for provider in providers:
+ for transform_type in provider.provided_transforms():
+ providers_dict[transform_type].append(provider)
+ spec = yaml_transform.preprocess(yaml.load(s, Loader=SafeLineLoader))
+ return Scope(
+ None, {},
+ transforms=spec['transforms'],
+ providers=providers_dict,
+ input_providers={})
+
+ def test_best_provider_based_on_input(self):
+ provider_Ax = TestProvider('A', 'xxx')
+ provider_Ay = TestProvider('A', 'yyy')
+ provider_Bx = TestProvider('B', 'xxz')
+ provider_By = TestProvider('B', 'yyz')
+ scope = self.create_scope(
+ '''
+ type: chain
+ transforms:
+ - type: A
+ - type: B
+ ''', [provider_Ax, provider_Ay, provider_Bx, provider_By])
+ self.assertEqual(scope.best_provider('B', [provider_Ax]), provider_Bx)
+ self.assertEqual(scope.best_provider('B', [provider_Ay]), provider_By)
+
+ def test_best_provider_based_on_followers(self):
+ close_provider = TestProvider('A', 'xxy')
+ far_provider = TestProvider('A', 'yyy')
+ following_provider = TestProvider('B', 'xxx')
+ scope = self.create_scope(
+ '''
+ type: chain
+ transforms:
+ - type: A
+ - type: B
+ ''', [far_provider, close_provider, following_provider])
+ self.assertEqual(scope.best_provider('A', []), close_provider)
+
+ def test_best_provider_based_on_multiple_followers(self):
+ close_provider = TestProvider('A', 'xxy')
+ provider_B = TestProvider('B', 'xxx')
+ # These are not quite as close as the two above.
+ far_provider = TestProvider('A', 'yyy')
+ provider_C = TestProvider('C', 'yzz')
+ scope = self.create_scope(
+ '''
+ type: composite
+ transforms:
+ - type: A
+ - type: B
+ input: A
+ - type: C
+ input: A
+ ''', [far_provider, close_provider, provider_B, provider_C])
+ self.assertEqual(scope.best_provider('A', []), close_provider)
+
+ def test_best_provider_based_on_distant_follower(self):
+ providers = [
+ # xxx and yyy vend both
+ TestProvider('A', 'xxx'),
+ TestProvider('A', 'yyy'),
+ TestProvider('B', 'xxx'),
+ TestProvider('B', 'yyy'),
+ TestProvider('C', 'xxx'),
+ TestProvider('C', 'yyy'),
+ # D and E are only provided by a single provider each.
+ TestProvider('D', 'xxx'),
+ TestProvider('E', 'yyy')
+ ]
+
+ # If D is the eventual destination, pick the xxx one.
+ scope = self.create_scope(
+ '''
+ type: chain
+ transforms:
+ - type: A
+ - type: B
+ - type: C
+ - type: D
+ ''',
+ providers)
+ self.assertEqual(scope.best_provider('A', []), providers[0])
+
+ # If instead E is the eventual destination, pick the yyy one.
+ scope = self.create_scope(
+ '''
+ type: chain
+ transforms:
+ - type: A
+ - type: B
+ - type: C
+ - type: E
+ ''',
+ providers)
+ self.assertEqual(scope.best_provider('A', []), providers[1])
+
+ # If we have D then E, stay with xxx as long as possible to only switch once
+ scope = self.create_scope(
+ '''
+ type: chain
+ transforms:
+ - type: A
+ - type: B
+ - type: C
+ - type: D
+ - type: E
+ ''',
+ providers)
+ self.assertEqual(scope.best_provider('A', []), providers[0])
+
+
class LightweightScopeTest(unittest.TestCase):
@staticmethod
def get_spec():
diff --git a/sdks/python/apache_beam/yaml/yaml_transform_test.py b/sdks/python/apache_beam/yaml/yaml_transform_test.py
index 9a540e3551ff..26baebec86e4 100644
--- a/sdks/python/apache_beam/yaml/yaml_transform_test.py
+++ b/sdks/python/apache_beam/yaml/yaml_transform_test.py
@@ -117,6 +117,27 @@ def test_chain_with_root(self):
''')
assert_that(result, equal_to([41, 43, 47, 53, 61, 71, 83, 97, 113, 131]))
+ def create_has_schema(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: chain
+ transforms:
+ - type: Create
+ config:
+ elements: [{a: 1, b: 'x'}, {a: 2, b: 'y'}]
+ - type: MapToFields
+ config:
+ language: python
+ fields:
+ repeated: a * b
+ - type: PyMap
+ config:
+ fn: "lambda x: x.repeated"
+ ''')
+ assert_that(result, equal_to(['x', 'yy']))
+
def test_implicit_flatten(self):
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
pickle_library='cloudpickle')) as p:
@@ -229,6 +250,23 @@ def test_name_is_ambiguous(self):
output: AnotherFilter
''')
+ def test_annotations(self):
+ t = LinearTransform(5, b=100)
+ annotations = t.annotations()
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: chain
+ transforms:
+ - type: Create
+ config:
+ elements: [0, 1, 2, 3]
+ - type: %r
+ config: %s
+ ''' % (annotations['yaml_type'], annotations['yaml_args']))
+ assert_that(result, equal_to([100, 105, 110, 115]))
+
class CreateTimestamped(beam.PTransform):
def __init__(self, elements):
@@ -610,6 +648,19 @@ def test_prefers_same_provider_class(self):
label='StartWith3')
+@beam.transforms.ptransform.annotate_yaml
+class LinearTransform(beam.PTransform):
+ """A transform used for testing annotate_yaml."""
+ def __init__(self, a, b):
+ self._a = a
+ self._b = b
+
+ def expand(self, pcoll):
+ a = self._a
+ b = self._b
+ return pcoll | beam.Map(lambda x: a * x + b)
+
+
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()
diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py
index d10056fea5b0..d57a77d326fb 100644
--- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py
+++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py
@@ -37,6 +37,7 @@
from apache_beam.yaml.yaml_transform import normalize_source_sink
from apache_beam.yaml.yaml_transform import only_element
from apache_beam.yaml.yaml_transform import pipeline_as_composite
+from apache_beam.yaml.yaml_transform import preprocess
from apache_beam.yaml.yaml_transform import preprocess_flattened_inputs
from apache_beam.yaml.yaml_transform import preprocess_windowing
from apache_beam.yaml.yaml_transform import push_windowing_to_roots
@@ -186,7 +187,7 @@ def test_expand_composite_transform_with_name(self):
- type: Create
config:
elements: [0,1,2]
- output:
+ output:
Create
'''
scope, spec = self.get_scope_by_spec(p, spec)
@@ -204,7 +205,7 @@ def test_expand_composite_transform_with_name_input(self):
input: input
config:
fn: 'lambda x: x*x'
- output:
+ output:
PyMap
'''
elements = p | beam.Create(range(3))
@@ -222,7 +223,7 @@ def test_expand_composite_transform_root(self):
- type: Create
config:
elements: [0,1,2]
- output:
+ output:
Create
'''
scope, spec = self.get_scope_by_spec(p, spec)
@@ -315,7 +316,7 @@ def test_chain_as_composite_with_outputs_override(self):
def test_chain_as_composite_with_input(self):
spec = '''
type: chain
- input:
+ input:
elements
transforms:
- type: PyMap
@@ -367,7 +368,7 @@ def test_normalize_source_sink_only_source(self):
- type: PyMap
config:
fn: 'lambda x: x*x'
-
+
'''
spec = yaml.load(spec, Loader=SafeLineLoader)
result = normalize_source_sink(spec)
@@ -499,10 +500,10 @@ def test_normalize_inputs_outputs(self):
expected = '''
type: PyMap
- input:
+ input:
input: [Create1, Create2]
fn: 'lambda x: x*x'
- output:
+ output:
output: Squared
'''
self.assertYaml(expected, result)
@@ -512,7 +513,7 @@ def test_normalize_inputs_outputs_dict(self):
type: PyMap
input: [Create1, Create2]
fn: 'lambda x: x*x'
- output:
+ output:
out1: Squared1
out2: Squared2
'''
@@ -521,10 +522,10 @@ def test_normalize_inputs_outputs_dict(self):
expected = '''
type: PyMap
- input:
+ input:
input: [Create1, Create2]
fn: 'lambda x: x*x'
- output:
+ output:
out1: Squared1
out2: Squared2
'''
@@ -610,13 +611,13 @@ def test_push_windowing_to_roots(self):
windowing:
type: fixed
size: 2
- __consumed_outputs:
+ __consumed_outputs:
- null
input: {}
output: {}
- type: PyMap
fn: 'lambda x: x*x'
- input:
+ input:
input: Create
output: {}
windowing:
@@ -646,7 +647,7 @@ def test_preprocess_windowing_custom_type(self):
input: Create
transforms:
- type: SumGlobally
- input:
+ input:
input: {result['transforms'][1]['__uuid__']}
output: {{}}
- type: WindowInto
@@ -696,10 +697,10 @@ def test_preprocess_windowing_composite_with_windowing_outer(self):
input: {}
output: {}
- type: SumGlobally
- input:
+ input:
input: Create
output: {}
- output:
+ output:
output: SumGlobally
'''
self.assertYaml(expected, result)
@@ -736,13 +737,13 @@ def test_preprocess_windowing_composite_with_windowing_on_input(self):
input: {}
output: {}
- type: SumGlobally
- input:
+ input:
input: Create
windowing:
type: fixed
size: 4
output: {}
- output:
+ output:
output: SumGlobally
'''
self.assertYaml(expected, result)
@@ -771,12 +772,14 @@ def test_preprocess_windowing_other_type_with_no_inputs(self):
output: {{}}
- type: WindowInto
name: WindowInto[None]
- input: {result['transforms'][0]["__uuid__"]}
+ input:
+ input: {result['transforms'][0]["__uuid__"]}
windowing:
type: fixed
size: 4
output: {result['transforms'][1]["__uuid__"]}
'''
+ self.maxDiff = 1e9
self.assertYaml(expected, result)
@@ -805,7 +808,7 @@ def test_preprocess_flattened_inputs_implicit(self):
input1: Create2
- type: PyMap
fn: 'lambda x: x*x'
- input:
+ input:
input: {result['transforms'][0]['__uuid__']}
output: {{}}
output: CreateTimestamped
@@ -839,7 +842,7 @@ def test_preprocess_flattened_inputs_explicit_flatten(self):
output: {}
- type: PyMap
fn: 'lambda x: x*x'
- input:
+ input:
input: Flatten
output: {}
output: CreateTimestamped
@@ -865,6 +868,17 @@ def test_ensure_transforms_have_types_error(self):
spec = yaml.load(spec, Loader=SafeLineLoader)
with self.assertRaisesRegex(ValueError, r"Missing type .*"):
ensure_transforms_have_types(spec)
+ with self.assertRaisesRegex(ValueError, r"Missing type .*"):
+ preprocess(spec)
+
+ def test_ensure_transforms_have_providers_error(self):
+ spec = '''
+ type: UnknownType
+ '''
+ spec = yaml.load(spec, Loader=SafeLineLoader)
+ with self.assertRaisesRegex(ValueError,
+ r"Unknown type or missing provider .*"):
+ preprocess(spec, known_transforms=['KnownType'])
def test_ensure_errors_consumed_unconsumed(self):
spec = '''
diff --git a/sdks/python/apache_beam/yaml/yaml_udf_test.py b/sdks/python/apache_beam/yaml/yaml_udf_test.py
new file mode 100644
index 000000000000..bb15cd494757
--- /dev/null
+++ b/sdks/python/apache_beam/yaml/yaml_udf_test.py
@@ -0,0 +1,242 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+import logging
+import os
+import shutil
+import tempfile
+import unittest
+
+import apache_beam as beam
+from apache_beam.io import localfilesystem
+from apache_beam.options import pipeline_options
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+from apache_beam.yaml.yaml_transform import YamlTransform
+
+
+class YamlUDFMappingTest(unittest.TestCase):
+ def __init__(self, method_name='runYamlMappingTest'):
+ super().__init__(method_name)
+ self.data = [
+ beam.Row(label='11a', conductor=11, rank=0),
+ beam.Row(label='37a', conductor=37, rank=1),
+ beam.Row(label='389a', conductor=389, rank=2),
+ ]
+
+ def setUp(self):
+ self.tmpdir = tempfile.mkdtemp()
+ self.fs = localfilesystem.LocalFileSystem(pipeline_options)
+
+ def tearDown(self):
+ shutil.rmtree(self.tmpdir)
+
+ def test_map_to_fields_filter_inline_js(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ '''
+ type: MapToFields
+ input: input
+ config:
+ language: javascript
+ fields:
+ label:
+ callable: "function label_map(x) {return x.label + 'x'}"
+ conductor:
+ callable: "function conductor_map(x) {return x.conductor + 1}"
+ keep:
+ callable: "function filter(x) {return x.rank > 0}"
+ ''')
+ assert_that(
+ result,
+ equal_to([
+ beam.Row(label='37ax', conductor=38),
+ beam.Row(label='389ax', conductor=390),
+ ]))
+
+ def test_map_to_fields_filter_inline_py(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ '''
+ type: MapToFields
+ input: input
+ config:
+ language: python
+ fields:
+ label:
+ callable: "lambda x: x.label + 'x'"
+ conductor:
+ callable: "lambda x: x.conductor + 1"
+ keep:
+ callable: "lambda x: x.rank > 0"
+ ''')
+ assert_that(
+ result,
+ equal_to([
+ beam.Row(label='37ax', conductor=38),
+ beam.Row(label='389ax', conductor=390),
+ ]))
+
+ def test_filter_inline_js(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ '''
+ type: Filter
+ input: input
+ config:
+ language: javascript
+ keep:
+ callable: "function filter(x) {return x.rank > 0}"
+ ''')
+ assert_that(
+ result,
+ equal_to([
+ beam.Row(label='37a', conductor=37, rank=1),
+ beam.Row(label='389a', conductor=389, rank=2),
+ ]))
+
+ def test_filter_inline_py(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ '''
+ type: Filter
+ input: input
+ config:
+ language: python
+ keep:
+ callable: "lambda x: x.rank > 0"
+ ''')
+ assert_that(
+ result,
+ equal_to([
+ beam.Row(label='37a', conductor=37, rank=1),
+ beam.Row(label='389a', conductor=389, rank=2),
+ ]))
+
+ def test_filter_expression_js(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ '''
+ type: Filter
+ input: input
+ config:
+ language: javascript
+ keep:
+ expression: "label.toUpperCase().indexOf('3') == -1 && conductor"
+ ''')
+ assert_that(
+ result, equal_to([
+ beam.Row(label='11a', conductor=11, rank=0),
+ ]))
+
+ def test_filter_expression_py(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ '''
+ type: Filter
+ input: input
+ config:
+ language: python
+ keep:
+ expression: "'3' not in label"
+ ''')
+ assert_that(
+ result, equal_to([
+ beam.Row(label='11a', conductor=11, rank=0),
+ ]))
+
+ def test_filter_inline_js_file(self):
+ data = '''
+ function f(x) {
+ return x.rank > 0
+ }
+
+ function g(x) {
+ return x.rank > 1
+ }
+ '''.replace(' ', '')
+
+ path = os.path.join(self.tmpdir, 'udf.js')
+ self.fs.create(path).write(data.encode('utf8'))
+
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ f'''
+ type: Filter
+ input: input
+ config:
+ language: javascript
+ keep:
+ path: {path}
+ name: "f"
+ ''')
+ assert_that(
+ result,
+ equal_to([
+ beam.Row(label='37a', conductor=37, rank=1),
+ beam.Row(label='389a', conductor=389, rank=2),
+ ]))
+
+ def test_filter_inline_py_file(self):
+ data = '''
+ def f(x):
+ return x.rank > 0
+
+ def g(x):
+ return x.rank > 1
+ '''.replace(' ', '')
+
+ path = os.path.join(self.tmpdir, 'udf.py')
+ self.fs.create(path).write(data.encode('utf8'))
+
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ elements = p | beam.Create(self.data)
+ result = elements | YamlTransform(
+ f'''
+ type: Filter
+ input: input
+ config:
+ language: python
+ keep:
+ path: {path}
+ name: "f"
+ ''')
+ assert_that(
+ result,
+ equal_to([
+ beam.Row(label='37a', conductor=37, rank=1),
+ beam.Row(label='389a', conductor=389, rank=2),
+ ]))
+
+
+if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.INFO)
+ unittest.main()
diff --git a/sdks/python/container/base_image_requirements_manual.txt b/sdks/python/container/base_image_requirements_manual.txt
index a1d80320d42d..e952b2126604 100644
--- a/sdks/python/container/base_image_requirements_manual.txt
+++ b/sdks/python/container/base_image_requirements_manual.txt
@@ -43,4 +43,3 @@ nose==1.3.7 # For Dataflow internal testing. TODO: remove this.
python-snappy;python_version<"3.11" # Optimizes execution of some Beam codepaths.
scipy
scikit-learn
-tensorflow>=2.12.0
diff --git a/sdks/python/container/boot.go b/sdks/python/container/boot.go
index e7b11daa3973..ded10a44204a 100644
--- a/sdks/python/container/boot.go
+++ b/sdks/python/container/boot.go
@@ -23,6 +23,7 @@ import (
"errors"
"flag"
"fmt"
+ "io"
"log"
"os"
"os/exec"
@@ -205,7 +206,7 @@ func launchSDKProcess() error {
}
}
- if setupErr := installSetupPackages(fileNames, dir, requirementsFiles); setupErr != nil {
+ if setupErr := installSetupPackages(ctx, logger, fileNames, dir, requirementsFiles); setupErr != nil {
fmtErr := fmt.Errorf("failed to install required packages: %v", setupErr)
// Send error message to logging service before returning up the call stack
logger.Errorf(ctx, fmtErr.Error())
@@ -272,6 +273,7 @@ func launchSDKProcess() error {
go func(workerId string) {
defer wg.Done()
+ bufLogger := tools.NewBufferedLogger(logger)
errorCount := 0
for {
childPids.mu.Lock()
@@ -280,7 +282,7 @@ func launchSDKProcess() error {
return
}
logger.Printf(ctx, "Executing Python (worker %v): python %v", workerId, strings.Join(args, " "))
- cmd := StartCommandEnv(map[string]string{"WORKER_ID": workerId}, "python", args...)
+ cmd := StartCommandEnv(map[string]string{"WORKER_ID": workerId}, os.Stdin, bufLogger, bufLogger, "python", args...)
childPids.v = append(childPids.v, cmd.Process.Pid)
childPids.mu.Unlock()
@@ -288,6 +290,7 @@ func launchSDKProcess() error {
// Retry on fatal errors, like OOMs and segfaults, not just
// DoFns throwing exceptions.
errorCount += 1
+ bufLogger.FlushAtError(ctx)
if errorCount < 4 {
logger.Warnf(ctx, "Python (worker %v) exited %v times: %v\nrestarting SDK process",
workerId, errorCount, err)
@@ -296,6 +299,7 @@ func launchSDKProcess() error {
workerId, errorCount, err)
}
} else {
+ bufLogger.FlushAtDebug(ctx)
logger.Printf(ctx, "Python (worker %v) exited.", workerId)
break
}
@@ -309,11 +313,11 @@ func launchSDKProcess() error {
// Start a command object in a new process group with the given arguments with
// additional environment variables. It attaches stdio to the child process.
// Returns the process handle.
-func StartCommandEnv(env map[string]string, prog string, args ...string) *exec.Cmd {
+func StartCommandEnv(env map[string]string, stdin io.Reader, stdout, stderr io.Writer, prog string, args ...string) *exec.Cmd {
cmd := exec.Command(prog, args...)
- cmd.Stdin = os.Stdin
- cmd.Stdout = os.Stdout
- cmd.Stderr = os.Stderr
+ cmd.Stdin = stdin
+ cmd.Stdout = stdout
+ cmd.Stderr = stderr
if env != nil {
cmd.Env = os.Environ()
for k, v := range env {
@@ -367,47 +371,42 @@ func setupAcceptableWheelSpecs() error {
return fmt.Errorf("cannot get parse Python version from %s", stdoutStderr)
}
pyVersion := fmt.Sprintf("%s%s", pyVersions[1], pyVersions[2])
- var wheelName string
- switch pyVersion {
- case "36", "37":
- wheelName = fmt.Sprintf("cp%s-cp%sm-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", pyVersion, pyVersion)
- default:
- wheelName = fmt.Sprintf("cp%s-cp%s-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", pyVersion, pyVersion)
- }
+ wheelName := fmt.Sprintf("cp%s-cp%s-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", pyVersion, pyVersion)
acceptableWhlSpecs = append(acceptableWhlSpecs, wheelName)
return nil
}
// installSetupPackages installs Beam SDK and user dependencies.
-func installSetupPackages(files []string, workDir string, requirementsFiles []string) error {
- log.Printf("Installing setup packages ...")
+func installSetupPackages(ctx context.Context, logger *tools.Logger, files []string, workDir string, requirementsFiles []string) error {
+ bufLogger := tools.NewBufferedLogger(logger)
+ bufLogger.Printf(ctx, "Installing setup packages ...")
if err := setupAcceptableWheelSpecs(); err != nil {
- log.Printf("Failed to setup acceptable wheel specs, leave it as empty: %v", err)
+ bufLogger.Printf(ctx, "Failed to setup acceptable wheel specs, leave it as empty: %v", err)
}
pkgName := "apache-beam"
isSdkInstalled := isPackageInstalled(pkgName)
if !isSdkInstalled {
- return fmt.Errorf("Apache Beam is not installed in the runtime environment. If you use a custom container image, you must install apache-beam package in the custom image using same version of Beam as in the pipeline submission environment. For more information, see: the https://beam.apache.org/documentation/runtime/environments/.")
+ return fmt.Errorf("Apache Beam is not installed in the runtime environment. If you use a custom container image, you must install apache-beam package in the custom image using same version of Beam as in the pipeline submission environment. For more information, see: the https://beam.apache.org/documentation/runtime/environments/")
}
// Install the Dataflow Python SDK and worker packages.
// We install the extra requirements in case of using the beam sdk. These are ignored by pip
// if the user is using an SDK that does not provide these.
- if err := installSdk(files, workDir, sdkSrcFile, acceptableWhlSpecs, false); err != nil {
+ if err := installSdk(ctx, logger, files, workDir, sdkSrcFile, acceptableWhlSpecs, false); err != nil {
return fmt.Errorf("failed to install SDK: %v", err)
}
// The staged files will not disappear due to restarts because workDir is a
// folder that is mapped to the host (and therefore survives restarts).
for _, f := range requirementsFiles {
- if err := pipInstallRequirements(files, workDir, f); err != nil {
+ if err := pipInstallRequirements(ctx, logger, files, workDir, f); err != nil {
return fmt.Errorf("failed to install requirements: %v", err)
}
}
- if err := installExtraPackages(files, extraPackagesFile, workDir); err != nil {
+ if err := installExtraPackages(ctx, logger, files, extraPackagesFile, workDir); err != nil {
return fmt.Errorf("failed to install extra packages: %v", err)
}
- if err := pipInstallPackage(files, workDir, workflowFile, false, true, nil); err != nil {
+ if err := pipInstallPackage(ctx, logger, files, workDir, workflowFile, false, true, nil); err != nil {
return fmt.Errorf("failed to install workflow: %v", err)
}
@@ -450,7 +449,7 @@ func processArtifactsInSetupOnlyMode() {
}
files[i] = filePayload.GetPath()
}
- if setupErr := installSetupPackages(files, workDir, []string{requirementsFile}); setupErr != nil {
+ if setupErr := installSetupPackages(context.Background(), nil, files, workDir, []string{requirementsFile}); setupErr != nil {
log.Fatalf("Failed to install required packages: %v", setupErr)
}
}
diff --git a/sdks/python/container/license_scripts/dep_urls_py.yaml b/sdks/python/container/license_scripts/dep_urls_py.yaml
index beea506ca91c..36efb36c321c 100644
--- a/sdks/python/container/license_scripts/dep_urls_py.yaml
+++ b/sdks/python/container/license_scripts/dep_urls_py.yaml
@@ -129,6 +129,8 @@ pip_dependencies:
notice: "https://raw.githubusercontent.com/apache/arrow/master/NOTICE.txt"
pyhamcrest:
license: "https://raw.githubusercontent.com/hamcrest/PyHamcrest/master/LICENSE.txt"
+ pyjsparser:
+ license: "https://github.com/PiotrDabkowski/pyjsparser/blob/master/LICENSE"
pymongo:
license: "https://raw.githubusercontent.com/mongodb/mongo-python-driver/master/LICENSE"
python-gflags:
diff --git a/sdks/python/container/piputil.go b/sdks/python/container/piputil.go
index 350bda049d9d..67488bdc39f7 100644
--- a/sdks/python/container/piputil.go
+++ b/sdks/python/container/piputil.go
@@ -18,24 +18,27 @@ package main
import (
"bufio"
"bytes"
+ "context"
"errors"
"fmt"
- "log"
"os"
"os/exec"
"path/filepath"
"strings"
+ "time"
+ "github.com/apache/beam/sdks/v2/go/container/tools"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/xlangx/expansionx"
"github.com/apache/beam/sdks/v2/go/pkg/beam/util/execx"
)
// pipInstallRequirements installs the given requirement, if present.
-func pipInstallRequirements(files []string, dir, name string) error {
+func pipInstallRequirements(ctx context.Context, logger *tools.Logger, files []string, dir, name string) error {
pythonVersion, err := expansionx.GetPythonVersion()
if err != nil {
return err
}
+ bufLogger := tools.NewBufferedLogger(logger)
for _, file := range files {
if file == name {
// We run the install process in two rounds in order to avoid as much
@@ -44,13 +47,19 @@ func pipInstallRequirements(files []string, dir, name string) error {
// used without following their dependencies.
args := []string{"-m", "pip", "install", "-q", "-r", filepath.Join(dir, name), "--no-cache-dir", "--disable-pip-version-check", "--no-index", "--no-deps", "--find-links", dir}
if err := execx.Execute(pythonVersion, args...); err != nil {
- fmt.Println("Some packages could not be installed solely from the requirements cache. Installing packages from PyPI.")
+ bufLogger.Printf(ctx, "Some packages could not be installed solely from the requirements cache. Installing packages from PyPI.")
}
// The second install round opens up the search for packages on PyPI and
// also installs dependencies. The key is that if all the packages have
// been installed in the first round then this command will be a no-op.
args = []string{"-m", "pip", "install", "-q", "-r", filepath.Join(dir, name), "--no-cache-dir", "--disable-pip-version-check", "--find-links", dir}
- return execx.Execute(pythonVersion, args...)
+ err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...)
+ if err != nil {
+ bufLogger.FlushAtError(ctx)
+ return err
+ }
+ bufLogger.FlushAtDebug(ctx)
+ return nil
}
}
return nil
@@ -68,12 +77,15 @@ func isPackageInstalled(pkgName string) bool {
return true
}
+const pipLogFlushInterval time.Duration = 15 * time.Second
+
// pipInstallPackage installs the given package, if present.
-func pipInstallPackage(files []string, dir, name string, force, optional bool, extras []string) error {
+func pipInstallPackage(ctx context.Context, logger *tools.Logger, files []string, dir, name string, force, optional bool, extras []string) error {
pythonVersion, err := expansionx.GetPythonVersion()
if err != nil {
return err
}
+ bufLogger := tools.NewBufferedLoggerWithFlushInterval(ctx, logger, pipLogFlushInterval)
for _, file := range files {
if file == name {
var packageSpec = name
@@ -97,19 +109,34 @@ func pipInstallPackage(files []string, dir, name string, force, optional bool, e
// installed version will match the package specified, the package itself
// will not be reinstalled, but its dependencies will now be resolved and
// installed if necessary. This achieves our goal outlined above.
- args := []string{"-m", "pip", "install", "-q", "--no-cache-dir", "--disable-pip-version-check", "--upgrade", "--force-reinstall", "--no-deps",
+ args := []string{"-m", "pip", "install", "--no-cache-dir", "--disable-pip-version-check", "--upgrade", "--force-reinstall", "--no-deps",
filepath.Join(dir, packageSpec)}
- err := execx.Execute(pythonVersion, args...)
+ err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...)
if err != nil {
+ bufLogger.FlushAtError(ctx)
return err
+ } else {
+ bufLogger.FlushAtDebug(ctx)
}
- args = []string{"-m", "pip", "install", "-q", "--no-cache-dir", "--disable-pip-version-check", filepath.Join(dir, packageSpec)}
- return execx.Execute(pythonVersion, args...)
+ args = []string{"-m", "pip", "install", "--no-cache-dir", "--disable-pip-version-check", filepath.Join(dir, packageSpec)}
+ err = execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...)
+ if err != nil {
+ bufLogger.FlushAtError(ctx)
+ return err
+ }
+ bufLogger.FlushAtDebug(ctx)
+ return nil
}
// Case when we do not perform a forced reinstall.
- args := []string{"-m", "pip", "install", "-q", "--no-cache-dir", "--disable-pip-version-check", filepath.Join(dir, packageSpec)}
- return execx.Execute(pythonVersion, args...)
+ args := []string{"-m", "pip", "install", "--no-cache-dir", "--disable-pip-version-check", filepath.Join(dir, packageSpec)}
+ err := execx.ExecuteEnvWithIO(nil, os.Stdin, bufLogger, bufLogger, pythonVersion, args...)
+ if err != nil {
+ bufLogger.FlushAtError(ctx)
+ return err
+ }
+ bufLogger.FlushAtDebug(ctx)
+ return nil
}
}
if optional {
@@ -120,7 +147,8 @@ func pipInstallPackage(files []string, dir, name string, force, optional bool, e
// installExtraPackages installs all the packages declared in the extra
// packages manifest file.
-func installExtraPackages(files []string, extraPackagesFile, dir string) error {
+func installExtraPackages(ctx context.Context, logger *tools.Logger, files []string, extraPackagesFile, dir string) error {
+ bufLogger := tools.NewBufferedLogger(logger)
// First check that extra packages manifest file is present.
for _, file := range files {
if file != extraPackagesFile {
@@ -138,8 +166,8 @@ func installExtraPackages(files []string, extraPackagesFile, dir string) error {
for s.Scan() {
extraPackage := s.Text()
- log.Printf("Installing extra package: %s", extraPackage)
- if err = pipInstallPackage(files, dir, extraPackage, true, false, nil); err != nil {
+ bufLogger.Printf(ctx, "Installing extra package: %s", extraPackage)
+ if err = pipInstallPackage(ctx, logger, files, dir, extraPackage, true, false, nil); err != nil {
return fmt.Errorf("failed to install extra package %s: %v", extraPackage, err)
}
}
@@ -148,12 +176,13 @@ func installExtraPackages(files []string, extraPackagesFile, dir string) error {
return nil
}
-func findBeamSdkWhl(files []string, acceptableWhlSpecs []string) string {
+func findBeamSdkWhl(ctx context.Context, logger *tools.Logger, files []string, acceptableWhlSpecs []string) string {
+ bufLogger := tools.NewBufferedLogger(logger)
for _, file := range files {
if strings.HasPrefix(file, "apache_beam") {
for _, s := range acceptableWhlSpecs {
if strings.HasSuffix(file, s) {
- log.Printf("Found Apache Beam SDK wheel: %v", file)
+ bufLogger.Printf(ctx, "Found Apache Beam SDK wheel: %v", file)
return file
}
}
@@ -167,17 +196,17 @@ func findBeamSdkWhl(files []string, acceptableWhlSpecs []string) string {
// assume that the pipleine was started with the Beam SDK found in the wheel
// file, and we try to install it. If not successful, we fall back to installing
// SDK from source tarball provided in sdkSrcFile.
-func installSdk(files []string, workDir string, sdkSrcFile string, acceptableWhlSpecs []string, required bool) error {
- sdkWhlFile := findBeamSdkWhl(files, acceptableWhlSpecs)
-
+func installSdk(ctx context.Context, logger *tools.Logger, files []string, workDir string, sdkSrcFile string, acceptableWhlSpecs []string, required bool) error {
+ sdkWhlFile := findBeamSdkWhl(ctx, logger, files, acceptableWhlSpecs)
+ bufLogger := tools.NewBufferedLogger(logger)
if sdkWhlFile != "" {
// by default, pip rejects to install wheel if same version already installed
isDev := strings.Contains(sdkWhlFile, ".dev")
- err := pipInstallPackage(files, workDir, sdkWhlFile, isDev, false, []string{"gcp"})
+ err := pipInstallPackage(ctx, logger, files, workDir, sdkWhlFile, isDev, false, []string{"gcp"})
if err == nil {
return nil
}
- log.Printf("Could not install Apache Beam SDK from a wheel: %v, proceeding to install SDK from source tarball.", err)
+ bufLogger.Printf(ctx, "Could not install Apache Beam SDK from a wheel: %v, proceeding to install SDK from source tarball.", err)
}
if !required {
_, err := os.Stat(filepath.Join(workDir, sdkSrcFile))
@@ -185,6 +214,6 @@ func installSdk(files []string, workDir string, sdkSrcFile string, acceptableWhl
return nil
}
}
- err := pipInstallPackage(files, workDir, sdkSrcFile, false, false, []string{"gcp"})
+ err := pipInstallPackage(ctx, logger, files, workDir, sdkSrcFile, false, false, []string{"gcp"})
return err
}
diff --git a/sdks/python/container/py310/base_image_requirements.txt b/sdks/python/container/py310/base_image_requirements.txt
index 5d88dae52505..340f3e1b9691 100644
--- a/sdks/python/container/py310/base_image_requirements.txt
+++ b/sdks/python/container/py310/base_image_requirements.txt
@@ -21,8 +21,6 @@
# https://s.apache.org/beam-python-dev-wiki
# Reach out to a committer if you need help.
-absl-py==1.4.0
-astunparse==1.6.3
attrs==23.1.0
beautifulsoup4==4.12.2
bs4==0.0.1
@@ -30,7 +28,7 @@ cachetools==5.3.1
certifi==2023.7.22
cffi==1.15.1
charset-normalizer==3.2.0
-click==8.1.6
+click==8.1.7
cloudpickle==2.2.1
crcmod==1.7
cryptography==41.0.3
@@ -40,74 +38,64 @@ dill==0.3.1.1
dnspython==2.4.2
docker==6.1.3
docopt==0.6.2
-exceptiongroup==1.1.2
+exceptiongroup==1.1.3
execnet==2.0.2
-fastavro==1.8.2
+fastavro==1.8.3
fasteners==0.18
-flatbuffers==23.5.26
freezegun==1.2.2
future==0.18.3
-gast==0.4.0
google-api-core==2.11.1
-google-api-python-client==2.96.0
+google-api-python-client==2.99.0
google-apitools==0.5.31
-google-auth==2.22.0
-google-auth-httplib2==0.1.0
-google-auth-oauthlib==1.0.0
-google-cloud-aiplatform==1.29.0
+google-auth==2.23.0
+google-auth-httplib2==0.1.1
+google-cloud-aiplatform==1.32.0
google-cloud-bigquery==3.11.4
google-cloud-bigquery-storage==2.22.0
google-cloud-bigtable==2.21.0
google-cloud-core==2.3.3
-google-cloud-datastore==2.17.0
+google-cloud-datastore==2.18.0
google-cloud-dlp==3.12.2
-google-cloud-language==2.10.1
-google-cloud-profiler==4.0.0
-google-cloud-pubsub==2.18.2
+google-cloud-language==2.11.0
+google-cloud-profiler==4.1.0
+google-cloud-pubsub==2.18.4
google-cloud-pubsublite==1.8.3
google-cloud-recommendations-ai==0.10.4
google-cloud-resource-manager==1.10.3
-google-cloud-spanner==3.40.0
+google-cloud-spanner==3.40.1
google-cloud-storage==2.10.0
google-cloud-videointelligence==2.11.3
google-cloud-vision==3.4.4
google-crc32c==1.5.0
-google-pasta==0.2.0
-google-resumable-media==2.5.0
+google-resumable-media==2.6.0
googleapis-common-protos==1.60.0
greenlet==2.0.2
grpc-google-iam-v1==0.12.6
-grpcio==1.56.2
-grpcio-status==1.56.2
+grpcio==1.58.0
+grpcio-status==1.58.0
guppy3==3.1.3
-h5py==3.9.0
hdfs==2.7.2
httplib2==0.22.0
-hypothesis==6.82.3
+hypothesis==6.84.3
idna==3.4
iniconfig==2.0.0
joblib==1.3.2
-keras==2.13.1
-libclang==16.0.6
-Markdown==3.4.4
-MarkupSafe==2.1.3
+Js2Py==0.74
mmh3==4.0.1
mock==5.1.0
nltk==3.8.1
nose==1.3.7
-numpy==1.24.3
+numpy==1.24.4
oauth2client==4.1.3
-oauthlib==3.2.2
objsize==0.6.1
-opt-einsum==3.3.0
-orjson==3.9.4
+orjson==3.9.7
overrides==6.5.0
packaging==23.1
pandas==1.5.3
parameterized==0.9.0
-pluggy==1.2.0
+pluggy==1.3.0
proto-plus==1.22.3
-protobuf==4.23.4
+protobuf==4.24.3
psycopg2-binary==2.9.7
pyarrow==11.0.0
pyasn1==0.5.0
@@ -115,45 +103,38 @@ pyasn1-modules==0.3.0
pycparser==2.21
pydot==1.4.2
PyHamcrest==2.0.4
-pymongo==4.4.1
+pyjsparser==2.7.1
+pymongo==4.5.0
PyMySQL==1.1.0
pyparsing==3.1.1
-pytest==7.4.0
+pytest==7.4.2
pytest-timeout==2.1.0
pytest-xdist==3.3.1
python-dateutil==2.8.2
python-snappy==0.6.1
-pytz==2023.3
+pytz==2023.3.post1
PyYAML==6.0.1
regex==2023.8.8
requests==2.31.0
requests-mock==1.11.0
-requests-oauthlib==1.3.1
rsa==4.9
scikit-learn==1.3.0
-scipy==1.11.1
+scipy==1.11.2
Shapely==1.8.5.post1
six==1.16.0
sortedcontainers==2.4.0
-soupsieve==2.4.1
+soupsieve==2.5
SQLAlchemy==1.4.49
sqlparse==0.4.4
-tenacity==8.2.2
-tensorboard==2.13.0
-tensorboard-data-server==0.7.1
-tensorflow==2.13.0
-tensorflow-cpu-aws==2.13.0;platform_machine=="aarch64"
-tensorflow-estimator==2.13.0
-tensorflow-io-gcs-filesystem==0.33.0
-termcolor==2.3.0
+tenacity==8.2.3
testcontainers==3.7.1
threadpoolctl==3.2.0
tomli==2.0.1
-tqdm==4.66.0
-typing_extensions==4.5.0
+tqdm==4.66.1
+typing_extensions==4.7.1
+tzlocal==5.0.1
uritemplate==4.1.1
urllib3==1.26.16
-websocket-client==1.6.1
-Werkzeug==2.3.6
+websocket-client==1.6.3
wrapt==1.15.0
-zstandard==0.21.0
\ No newline at end of file
+zstandard==0.21.0
diff --git a/sdks/python/container/py311/base_image_requirements.txt b/sdks/python/container/py311/base_image_requirements.txt
index e3f4b0b93930..cb1637c2eb12 100644
--- a/sdks/python/container/py311/base_image_requirements.txt
+++ b/sdks/python/container/py311/base_image_requirements.txt
@@ -21,8 +21,6 @@
# https://s.apache.org/beam-python-dev-wiki
# Reach out to a committer if you need help.
-absl-py==1.4.0
-astunparse==1.6.3
attrs==23.1.0
beautifulsoup4==4.12.2
bs4==0.0.1
@@ -30,7 +28,7 @@ cachetools==5.3.1
certifi==2023.7.22
cffi==1.15.1
charset-normalizer==3.2.0
-click==8.1.6
+click==8.1.7
cloudpickle==2.2.1
crcmod==1.7
cryptography==41.0.3
@@ -41,70 +39,60 @@ dnspython==2.4.2
docker==6.1.3
docopt==0.6.2
execnet==2.0.2
-fastavro==1.8.2
+fastavro==1.8.3
fasteners==0.18
-flatbuffers==23.5.26
freezegun==1.2.2
future==0.18.3
-gast==0.4.0
google-api-core==2.11.1
google-apitools==0.5.31
-google-auth==2.22.0
-google-auth-httplib2==0.1.0
-google-auth-oauthlib==1.0.0
-google-cloud-aiplatform==1.29.0
+google-auth==2.23.0
+google-auth-httplib2==0.1.1
+google-cloud-aiplatform==1.32.0
google-cloud-bigquery==3.11.4
google-cloud-bigquery-storage==2.22.0
google-cloud-bigtable==2.21.0
google-cloud-core==2.3.3
-google-cloud-datastore==2.17.0
+google-cloud-datastore==2.18.0
google-cloud-dlp==3.12.2
-google-cloud-language==2.10.1
-google-cloud-pubsub==2.18.2
+google-cloud-language==2.11.0
+google-cloud-pubsub==2.18.4
google-cloud-pubsublite==1.8.3
google-cloud-recommendations-ai==0.10.4
google-cloud-resource-manager==1.10.3
-google-cloud-spanner==3.40.0
+google-cloud-spanner==3.40.1
google-cloud-storage==2.10.0
google-cloud-videointelligence==2.11.3
google-cloud-vision==3.4.4
google-crc32c==1.5.0
-google-pasta==0.2.0
-google-resumable-media==2.5.0
+google-resumable-media==2.6.0
googleapis-common-protos==1.60.0
greenlet==2.0.2
grpc-google-iam-v1==0.12.6
-grpcio==1.56.2
-grpcio-status==1.56.2
+grpcio==1.58.0
+grpcio-status==1.58.0
guppy3==3.1.3
-h5py==3.9.0
hdfs==2.7.2
httplib2==0.22.0
-hypothesis==6.82.3
+hypothesis==6.84.3
idna==3.4
iniconfig==2.0.0
joblib==1.3.2
-keras==2.13.1
-libclang==16.0.6
-Markdown==3.4.4
-MarkupSafe==2.1.3
+Js2Py==0.74
mmh3==4.0.1
mock==5.1.0
nltk==3.8.1
nose==1.3.7
-numpy==1.24.3
+numpy==1.24.4
oauth2client==4.1.3
-oauthlib==3.2.2
objsize==0.6.1
-opt-einsum==3.3.0
-orjson==3.9.4
+orjson==3.9.7
overrides==6.5.0
packaging==23.1
pandas==1.5.3
parameterized==0.9.0
-pluggy==1.2.0
+pluggy==1.3.0
proto-plus==1.22.3
-protobuf==4.23.4
+protobuf==4.24.3
psycopg2-binary==2.9.7
pyarrow==11.0.0
pyasn1==0.5.0
@@ -112,42 +100,35 @@ pyasn1-modules==0.3.0
pycparser==2.21
pydot==1.4.2
PyHamcrest==2.0.4
-pymongo==4.4.1
+pyjsparser==2.7.1
+pymongo==4.5.0
PyMySQL==1.1.0
pyparsing==3.1.1
-pytest==7.4.0
+pytest==7.4.2
pytest-timeout==2.1.0
pytest-xdist==3.3.1
python-dateutil==2.8.2
-pytz==2023.3
+pytz==2023.3.post1
PyYAML==6.0.1
regex==2023.8.8
requests==2.31.0
requests-mock==1.11.0
-requests-oauthlib==1.3.1
rsa==4.9
scikit-learn==1.3.0
-scipy==1.11.1
+scipy==1.11.2
Shapely==1.8.5.post1
six==1.16.0
sortedcontainers==2.4.0
-soupsieve==2.4.1
+soupsieve==2.5
SQLAlchemy==1.4.49
sqlparse==0.4.4
-tenacity==8.2.2
-tensorboard==2.13.0
-tensorboard-data-server==0.7.1
-tensorflow==2.13.0
-tensorflow-cpu-aws==2.13.0;platform_machine=="aarch64"
-tensorflow-estimator==2.13.0
-tensorflow-io-gcs-filesystem==0.33.0
-termcolor==2.3.0
+tenacity==8.2.3
testcontainers==3.7.1
threadpoolctl==3.2.0
-tqdm==4.66.0
-typing_extensions==4.5.0
+tqdm==4.66.1
+typing_extensions==4.7.1
+tzlocal==5.0.1
urllib3==1.26.16
-websocket-client==1.6.1
-Werkzeug==2.3.6
+websocket-client==1.6.3
wrapt==1.15.0
-zstandard==0.21.0
\ No newline at end of file
+zstandard==0.21.0
diff --git a/sdks/python/container/py38/base_image_requirements.txt b/sdks/python/container/py38/base_image_requirements.txt
index 60a423c3f373..3d59060cd3ee 100644
--- a/sdks/python/container/py38/base_image_requirements.txt
+++ b/sdks/python/container/py38/base_image_requirements.txt
@@ -21,16 +21,15 @@
# https://s.apache.org/beam-python-dev-wiki
# Reach out to a committer if you need help.
-absl-py==1.4.0
-astunparse==1.6.3
attrs==23.1.0
+backports.zoneinfo==0.2.1
beautifulsoup4==4.12.2
bs4==0.0.1
cachetools==5.3.1
certifi==2023.7.22
cffi==1.15.1
charset-normalizer==3.2.0
-click==8.1.6
+click==8.1.7
cloudpickle==2.2.1
crcmod==1.7
cryptography==41.0.3
@@ -40,75 +39,64 @@ dill==0.3.1.1
dnspython==2.4.2
docker==6.1.3
docopt==0.6.2
-exceptiongroup==1.1.2
+exceptiongroup==1.1.3
execnet==2.0.2
-fastavro==1.8.2
+fastavro==1.8.3
fasteners==0.18
-flatbuffers==23.5.26
freezegun==1.2.2
future==0.18.3
-gast==0.4.0
google-api-core==2.11.1
-google-api-python-client==2.96.0
+google-api-python-client==2.99.0
google-apitools==0.5.31
-google-auth==2.22.0
-google-auth-httplib2==0.1.0
-google-auth-oauthlib==1.0.0
-google-cloud-aiplatform==1.29.0
+google-auth==2.23.0
+google-auth-httplib2==0.1.1
+google-cloud-aiplatform==1.32.0
google-cloud-bigquery==3.11.4
google-cloud-bigquery-storage==2.22.0
google-cloud-bigtable==2.21.0
google-cloud-core==2.3.3
-google-cloud-datastore==2.17.0
+google-cloud-datastore==2.18.0
google-cloud-dlp==3.12.2
-google-cloud-language==2.10.1
-google-cloud-profiler==4.0.0
-google-cloud-pubsub==2.18.2
+google-cloud-language==2.11.0
+google-cloud-profiler==4.1.0
+google-cloud-pubsub==2.18.4
google-cloud-pubsublite==1.8.3
google-cloud-recommendations-ai==0.10.4
google-cloud-resource-manager==1.10.3
-google-cloud-spanner==3.40.0
+google-cloud-spanner==3.40.1
google-cloud-storage==2.10.0
google-cloud-videointelligence==2.11.3
google-cloud-vision==3.4.4
google-crc32c==1.5.0
-google-pasta==0.2.0
-google-resumable-media==2.5.0
+google-resumable-media==2.6.0
googleapis-common-protos==1.60.0
greenlet==2.0.2
grpc-google-iam-v1==0.12.6
-grpcio==1.56.2
-grpcio-status==1.56.2
+grpcio==1.58.0
+grpcio-status==1.58.0
guppy3==3.1.3
-h5py==3.9.0
hdfs==2.7.2
httplib2==0.22.0
-hypothesis==6.82.3
+hypothesis==6.84.3
idna==3.4
-importlib-metadata==6.8.0
iniconfig==2.0.0
joblib==1.3.2
-keras==2.13.1
-libclang==16.0.6
-Markdown==3.4.4
-MarkupSafe==2.1.3
+Js2Py==0.74
mmh3==4.0.1
mock==5.1.0
nltk==3.8.1
nose==1.3.7
-numpy==1.24.3
+numpy==1.24.4
oauth2client==4.1.3
-oauthlib==3.2.2
objsize==0.6.1
-opt-einsum==3.3.0
-orjson==3.9.4
+orjson==3.9.7
overrides==6.5.0
packaging==23.1
pandas==1.5.3
parameterized==0.9.0
-pluggy==1.2.0
+pluggy==1.3.0
proto-plus==1.22.3
-protobuf==4.23.4
+protobuf==4.24.3
psycopg2-binary==2.9.7
pyarrow==11.0.0
pyasn1==0.5.0
@@ -116,46 +104,38 @@ pyasn1-modules==0.3.0
pycparser==2.21
pydot==1.4.2
PyHamcrest==2.0.4
-pymongo==4.4.1
+pyjsparser==2.7.1
+pymongo==4.5.0
PyMySQL==1.1.0
pyparsing==3.1.1
-pytest==7.4.0
+pytest==7.4.2
pytest-timeout==2.1.0
pytest-xdist==3.3.1
python-dateutil==2.8.2
python-snappy==0.6.1
-pytz==2023.3
+pytz==2023.3.post1
PyYAML==6.0.1
regex==2023.8.8
requests==2.31.0
requests-mock==1.11.0
-requests-oauthlib==1.3.1
rsa==4.9
scikit-learn==1.3.0
scipy==1.10.1
Shapely==1.8.5.post1
six==1.16.0
sortedcontainers==2.4.0
-soupsieve==2.4.1
+soupsieve==2.5
SQLAlchemy==1.4.49
sqlparse==0.4.4
-tenacity==8.2.2
-tensorboard==2.13.0
-tensorboard-data-server==0.7.1
-tensorflow==2.13.0
-tensorflow-cpu-aws==2.13.0;platform_machine=="aarch64"
-tensorflow-estimator==2.13.0
-tensorflow-io-gcs-filesystem==0.33.0
-termcolor==2.3.0
+tenacity==8.2.3
testcontainers==3.7.1
threadpoolctl==3.2.0
tomli==2.0.1
-tqdm==4.66.0
-typing_extensions==4.5.0
+tqdm==4.66.1
+typing_extensions==4.7.1
+tzlocal==5.0.1
uritemplate==4.1.1
urllib3==1.26.16
-websocket-client==1.6.1
-Werkzeug==2.3.6
+websocket-client==1.6.3
wrapt==1.15.0
-zipp==3.16.2
-zstandard==0.21.0
\ No newline at end of file
+zstandard==0.21.0
diff --git a/sdks/python/container/py39/base_image_requirements.txt b/sdks/python/container/py39/base_image_requirements.txt
index e65b4c8e1430..6342cfe6edc1 100644
--- a/sdks/python/container/py39/base_image_requirements.txt
+++ b/sdks/python/container/py39/base_image_requirements.txt
@@ -21,8 +21,6 @@
# https://s.apache.org/beam-python-dev-wiki
# Reach out to a committer if you need help.
-absl-py==1.4.0
-astunparse==1.6.3
attrs==23.1.0
beautifulsoup4==4.12.2
bs4==0.0.1
@@ -30,7 +28,7 @@ cachetools==5.3.1
certifi==2023.7.22
cffi==1.15.1
charset-normalizer==3.2.0
-click==8.1.6
+click==8.1.7
cloudpickle==2.2.1
crcmod==1.7
cryptography==41.0.3
@@ -40,75 +38,64 @@ dill==0.3.1.1
dnspython==2.4.2
docker==6.1.3
docopt==0.6.2
-exceptiongroup==1.1.2
+exceptiongroup==1.1.3
execnet==2.0.2
-fastavro==1.8.2
+fastavro==1.8.3
fasteners==0.18
-flatbuffers==23.5.26
freezegun==1.2.2
future==0.18.3
-gast==0.4.0
google-api-core==2.11.1
-google-api-python-client==2.96.0
+google-api-python-client==2.99.0
google-apitools==0.5.31
-google-auth==2.22.0
-google-auth-httplib2==0.1.0
-google-auth-oauthlib==1.0.0
-google-cloud-aiplatform==1.29.0
+google-auth==2.23.0
+google-auth-httplib2==0.1.1
+google-cloud-aiplatform==1.32.0
google-cloud-bigquery==3.11.4
google-cloud-bigquery-storage==2.22.0
google-cloud-bigtable==2.21.0
google-cloud-core==2.3.3
-google-cloud-datastore==2.17.0
+google-cloud-datastore==2.18.0
google-cloud-dlp==3.12.2
-google-cloud-language==2.10.1
-google-cloud-profiler==4.0.0
-google-cloud-pubsub==2.18.2
+google-cloud-language==2.11.0
+google-cloud-profiler==4.1.0
+google-cloud-pubsub==2.18.4
google-cloud-pubsublite==1.8.3
google-cloud-recommendations-ai==0.10.4
google-cloud-resource-manager==1.10.3
-google-cloud-spanner==3.40.0
+google-cloud-spanner==3.40.1
google-cloud-storage==2.10.0
google-cloud-videointelligence==2.11.3
google-cloud-vision==3.4.4
google-crc32c==1.5.0
-google-pasta==0.2.0
-google-resumable-media==2.5.0
+google-resumable-media==2.6.0
googleapis-common-protos==1.60.0
greenlet==2.0.2
grpc-google-iam-v1==0.12.6
-grpcio==1.56.2
-grpcio-status==1.56.2
+grpcio==1.58.0
+grpcio-status==1.58.0
guppy3==3.1.3
-h5py==3.9.0
hdfs==2.7.2
httplib2==0.22.0
-hypothesis==6.82.3
+hypothesis==6.84.3
idna==3.4
-importlib-metadata==6.8.0
iniconfig==2.0.0
joblib==1.3.2
-keras==2.13.1
-libclang==16.0.6
-Markdown==3.4.4
-MarkupSafe==2.1.3
+Js2Py==0.74
mmh3==4.0.1
mock==5.1.0
nltk==3.8.1
nose==1.3.7
-numpy==1.24.3
+numpy==1.24.4
oauth2client==4.1.3
-oauthlib==3.2.2
objsize==0.6.1
-opt-einsum==3.3.0
-orjson==3.9.4
+orjson==3.9.7
overrides==6.5.0
packaging==23.1
pandas==1.5.3
parameterized==0.9.0
-pluggy==1.2.0
+pluggy==1.3.0
proto-plus==1.22.3
-protobuf==4.23.4
+protobuf==4.24.3
psycopg2-binary==2.9.7
pyarrow==11.0.0
pyasn1==0.5.0
@@ -116,46 +103,38 @@ pyasn1-modules==0.3.0
pycparser==2.21
pydot==1.4.2
PyHamcrest==2.0.4
-pymongo==4.4.1
+pyjsparser==2.7.1
+pymongo==4.5.0
PyMySQL==1.1.0
pyparsing==3.1.1
-pytest==7.4.0
+pytest==7.4.2
pytest-timeout==2.1.0
pytest-xdist==3.3.1
python-dateutil==2.8.2
python-snappy==0.6.1
-pytz==2023.3
+pytz==2023.3.post1
PyYAML==6.0.1
regex==2023.8.8
requests==2.31.0
requests-mock==1.11.0
-requests-oauthlib==1.3.1
rsa==4.9
scikit-learn==1.3.0
-scipy==1.11.1
+scipy==1.11.2
Shapely==1.8.5.post1
six==1.16.0
sortedcontainers==2.4.0
-soupsieve==2.4.1
+soupsieve==2.5
SQLAlchemy==1.4.49
sqlparse==0.4.4
-tenacity==8.2.2
-tensorboard==2.13.0
-tensorboard-data-server==0.7.1
-tensorflow==2.13.0
-tensorflow-cpu-aws==2.13.0;platform_machine=="aarch64"
-tensorflow-estimator==2.13.0
-tensorflow-io-gcs-filesystem==0.33.0
-termcolor==2.3.0
+tenacity==8.2.3
testcontainers==3.7.1
threadpoolctl==3.2.0
tomli==2.0.1
-tqdm==4.66.0
-typing_extensions==4.5.0
+tqdm==4.66.1
+typing_extensions==4.7.1
+tzlocal==5.0.1
uritemplate==4.1.1
urllib3==1.26.16
-websocket-client==1.6.1
-Werkzeug==2.3.6
+websocket-client==1.6.3
wrapt==1.15.0
-zipp==3.16.2
-zstandard==0.21.0
\ No newline at end of file
+zstandard==0.21.0
diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh
index 06ea4dc3622c..06ad06320fcf 100755
--- a/sdks/python/scripts/generate_pydoc.sh
+++ b/sdks/python/scripts/generate_pydoc.sh
@@ -131,7 +131,9 @@ release = version
autoclass_content = 'both'
autodoc_inherit_docstrings = False
autodoc_member_order = 'bysource'
-autodoc_mock_imports = ["tensorrt", "cuda", "torch", "onnxruntime", "onnx", "tensorflow", "tensorflow_hub", "tensorflow_transform", "tensorflow_metadata"]
+autodoc_mock_imports = ["tensorrt", "cuda", "torch",
+ "onnxruntime", "onnx", "tensorflow", "tensorflow_hub",
+ "tensorflow_transform", "tensorflow_metadata", "transformers"]
# Allow a special section for documenting DataFrame API
napoleon_custom_sections = ['Differences from pandas']
diff --git a/sdks/python/scripts/run_snapshot_publish.sh b/sdks/python/scripts/run_snapshot_publish.sh
index 5cdde16f53a2..6379e6f21084 100755
--- a/sdks/python/scripts/run_snapshot_publish.sh
+++ b/sdks/python/scripts/run_snapshot_publish.sh
@@ -18,7 +18,7 @@
BUCKET=gs://beam-python-nightly-snapshots
-VERSION=$(awk '/__version__/{print $3}' $WORKSPACE/src/sdks/python/apache_beam/version.py)
+VERSION=$(awk '/__version__/{print $3}' $WORKSPACE/sdks/python/apache_beam/version.py)
VERSION=$(echo $VERSION | cut -c 2- | rev | cut -c 2- | rev)
time=$(date +"%Y-%m-%dT%H:%M:%S")
SNAPSHOT="apache-beam-$VERSION-$time.zip"
@@ -28,7 +28,7 @@ DEP_SNAPSHOT_FILE_NAME="beam-py-requirements-$time.txt"
# Snapshots are built by Gradle task :sdks:python:depSnapshot
# and located under Gradle build directory.
-cd $WORKSPACE/src/sdks/python/build
+cd $WORKSPACE/sdks/python/build
# Rename the file to be apache-beam-{VERSION}-{datetime}.zip
for file in "apache-beam-$VERSION*.zip"; do
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 042d165f83d8..cadc4f34c86d 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -230,7 +230,7 @@ def get_portability_package_data():
language_level=3),
install_requires=[
'crcmod>=1.7,<2.0',
- 'orjson<4.0',
+ 'orjson>=3.9.7,<4',
# Dill doesn't have forwards-compatibility guarantees within minor
# version. Pickles created with a new version of dill may not unpickle
# using older version of dill. It is best to use the same version of
@@ -247,6 +247,7 @@ def get_portability_package_data():
'grpcio>=1.33.1,!=1.48.0,<2',
'hdfs>=2.1.0,<3.0.0',
'httplib2>=0.8,<0.23.0',
+ 'js2py>=0.74,<1',
# numpy can have breaking changes in minor versions.
# Use a strict upper bound.
'numpy>=1.14.3,<1.25.0', # Update build-requirements.txt as well.
@@ -254,12 +255,17 @@ def get_portability_package_data():
'packaging>=22.0',
'pymongo>=3.8.0,<5.0.0',
'proto-plus>=1.7.1,<2',
- # use a tighter upper bound in protobuf dependency
- # to make sure the minor version at job submission
+ # 1. Use a tighter upper bound in protobuf dependency to make sure
+ # the minor version at job submission
# does not exceed the minor version at runtime.
# To avoid depending on an old dependency, update the minor version on
# every Beam release, see: https://github.com/apache/beam/issues/25590
- 'protobuf>=3.20.3,<4.24.0',
+
+ # 2. Allow latest protobuf 3 version as a courtesy to some customers.
+ #
+ # 3. Exclude protobuf 4 versions that leak memory, see:
+ # https://github.com/apache/beam/issues/28246
+ 'protobuf>=3.20.3,<4.25.0,!=4.0.*,!=4.21.*,!=4.22.0,!=4.23.*,!=4.24.0,!=4.24.1,!=4.24.2', # pylint: disable=line-too-long
'pydot>=1.2.0,<2',
'python-dateutil>=2.8.0,<3',
'pytz>=2018.3',
diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini
index 1dc605595d55..1caf25caf080 100644
--- a/sdks/python/tox.ini
+++ b/sdks/python/tox.ini
@@ -31,7 +31,7 @@ select = E3
# https://github.com/apache/beam/issues/25668
pip_pre = True
# allow apps that support color to use it.
-passenv=TERM
+passenv=TERM,CLOUDSDK_CONFIG
# Set [] options for pip installation of apache-beam tarball.
extras = test,dataframe
# Don't warn that these commands aren't installed.
@@ -165,7 +165,7 @@ deps =
torch
xgboost<=1.7.6
datatable==1.0.0
- transformers<=4.31.0
+ transformers
commands =
time {toxinidir}/scripts/generate_pydoc.sh
diff --git a/website/www/site/content/en/blog/beam-2.47.0.md b/website/www/site/content/en/blog/beam-2.47.0.md
index dd962d6824fb..eaf99ac7122e 100644
--- a/website/www/site/content/en/blog/beam-2.47.0.md
+++ b/website/www/site/content/en/blog/beam-2.47.0.md
@@ -66,6 +66,8 @@ For more information on changes in 2.47.0, check out the [detailed release notes
### Known Issues
* BigQueryIO Storage API write with autoUpdateSchema may cause data corruption for Beam SDKs 2.45.0 - 2.47.0 (inclusive) ([#26789](https://github.com/apache/beam/issues/26789))
+* Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246).
+
## List of Contributors
@@ -195,4 +197,4 @@ tvalentyn
xianhualiu
-zhangskz
\ No newline at end of file
+zhangskz
diff --git a/website/www/site/content/en/blog/beam-2.48.0.md b/website/www/site/content/en/blog/beam-2.48.0.md
index dc6b4daaeda9..11b7f88e99c8 100644
--- a/website/www/site/content/en/blog/beam-2.48.0.md
+++ b/website/www/site/content/en/blog/beam-2.48.0.md
@@ -66,6 +66,8 @@ For more information on changes in 2.48.0, check out the [detailed release notes
## Known Issues
* PubsubIO writes will throw *SizeLimitExceededException* for any message above 100 bytes, when used in batch (bounded) mode. (Java) ([#27000](https://github.com/apache/beam/issues/27000)).
+* Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246).
+
## List of Contributors
@@ -201,4 +203,4 @@ liferoad
mokamoka03210120
-psolomin
\ No newline at end of file
+psolomin
diff --git a/website/www/site/content/en/blog/beam-2.49.0.md b/website/www/site/content/en/blog/beam-2.49.0.md
index 621637d655f6..595b8e71253b 100644
--- a/website/www/site/content/en/blog/beam-2.49.0.md
+++ b/website/www/site/content/en/blog/beam-2.49.0.md
@@ -50,6 +50,8 @@ For more information on changes in 2.49.0, check out the [detailed release notes
### Known Issues
+* Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246).
+
## List of Contributors
diff --git a/website/www/site/content/en/blog/beam-2.50.0.md b/website/www/site/content/en/blog/beam-2.50.0.md
index ecb0098ebb36..4cfddd6167a6 100644
--- a/website/www/site/content/en/blog/beam-2.50.0.md
+++ b/website/www/site/content/en/blog/beam-2.50.0.md
@@ -77,10 +77,12 @@ For more information on changes in 2.50.0, check out the [detailed release notes
* Fixed DirectRunner bug in Python SDK where GroupByKey gets empty PCollection and fails when pipeline option `direct_num_workers!=1`.([#27373](https://github.com/apache/beam/pull/27373))
* Fixed BigQuery I/O bug when estimating size on queries that utilize row-level security ([#27474](https://github.com/apache/beam/pull/27474))
+* Beam Python containers rely on a version of Debian/aom that has several security vulnerabilities: [CVE-2021-30474](https://nvd.nist.gov/vuln/detail/CVE-2021-30474), [CVE-2021-30475](https://nvd.nist.gov/vuln/detail/CVE-2021-30475), [CVE-2021-30473](https://nvd.nist.gov/vuln/detail/CVE-2021-30473), [CVE-2020-36133](https://nvd.nist.gov/vuln/detail/CVE-2020-36133), [CVE-2020-36131](https://nvd.nist.gov/vuln/detail/CVE-2020-36131), [CVE-2020-36130](https://nvd.nist.gov/vuln/detail/CVE-2020-36130), and [CVE-2020-36135](https://nvd.nist.gov/vuln/detail/CVE-2020-36135).
## Known Issues
-* TBD
+* Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246).
+* Python Pipelines using BigQuery IO or `orjson` dependency might experience segmentation faults or get stuck: [#28318](https://github.com/apache/beam/issues/28318).
## List of Contributors
diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md
index 20da8fbffa6a..964d2bfa7050 100644
--- a/website/www/site/content/en/contribute/release-guide.md
+++ b/website/www/site/content/en/contribute/release-guide.md
@@ -37,23 +37,24 @@ This guide complements the foundation-wide [Product Release Policy](https://www.
### Overview
-
+
The release process consists of several steps:
1. Decide to release
1. Prepare for the release
-1. Update base image dependencies for Python container images
-1. Investigate performance regressions
-1. Create a release branch
-1. Verify release branch
1. Build a release candidate
+ 1. Create a release branch
+ 1. Verify release branch
+ 1. Tag a release candidate
+ 1. Build release artifacts
1. Vote on the release candidate
-1. During vote process, run validation tests
+ * During vote process, run validation tests
1. If necessary, fix any issues and go back to step 3.
1. Finalize the release
1. Promote the release
+**********
## 1. Decide to release
@@ -70,34 +71,86 @@ That said, if you are a committer interested in serving the community in this wa
### Checklist to proceed to the next step
1. Community agrees to release
-1. Community selects a Release Manager
+1. Community selects a committer as Release Manager
**********
-## 2. Prepare for the release
+## 2. Prepare for the release (~1 week before branch cut)
Before your first release, you should perform one-time configuration steps.
- This will set up your security keys for signing the release and access to various release repositories.
+This will set up your security keys for signing the release and access to various release repositories.
To prepare for each release, you should audit the project status in the GitHub issue tracker, and do necessary bookkeeping.
-Finally, you should create a release branch from which individual release candidates will be built.
+Finally, create a release branch from which individual release candidates will be built.
__NOTE__: If you are using [GitHub two-factor authentication](https://help.github.com/articles/securing-your-account-with-two-factor-authentication-2fa/) and haven't configure HTTPS access,
please follow [the guide](https://help.github.com/articles/creating-a-personal-access-token-for-the-command-line/) to configure command line access.
+### Create a new milestone in GitHub for the next release
+
+When contributors resolve an issue in GitHub, they are tagging it with a release that will contain their changes.
+With the release currently underway, new issues should be resolved against a subsequent future release.
+Therefore, you should create a release item for this subsequent release, as follows:
+
+1. In GitHub, navigate to [`Issues > Milestones > New Milestone`](https://github.com/apache/beam/milestones).
+1. Add a new release. Choose the next minor version number after the version currently underway, select the next release due date (generally 6 weeks from today’s date) as the `Start Date`, and choose `Create Milestone`.
+1. At the end of the release, go to the same page and mark the recently released version as closed.
### Accounts
Please have these credentials ready at hand, you will likely need to enter them multiple times:
-* GPG pass phrase (see the next section);
* Apache ID and Password;
-* GitHub ID and Password.
-* DockerHub ID and Password. (You should be a member of maintainer team; email at dev@ if you are not.)
-* Account to access to apache-beam-testing Google Cloud Platform project. The account must have permissions to start Cloud Build triggers. Required for Playground environment update. (E-mail to pabloem@google.com to request access)
+* GitHub ID, Password, and Personal Access Token.
+* PyPi account and apitoken
+* DockerHub ID and Password with beam maintainer access
+* GPG pass phrase & 16-digit key ID
+* Access to Beam's Apache Nexus repository
+* Account to access to apache-beam-testing Google Cloud Platform project. The account must have permissions to start Cloud Build triggers. Required for Playground environment update. (E-mail at dev@ mailing list to request access)
+
+If you don't have a given credential, follow the 'one-time' instructions below.
+
+**********
### One-time setup instructions
+#### Apache ID and Password
+
+This is your Apache committer user name and password. You selected these when you became an Apache Beam Committer.
+
+#### Github ID, Password, and Personal Access Token
+
+For some scripts, you need a Personal Access Token with `repo` and `workflow` permissions.
+They can be generated from this page: https://github.com/settings/tokens.
+See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens for details.
+
+#### Register to PyPI
+
+Release manager needs to have an account with PyPI.
+If you need one, [register at PyPI](https://pypi.python.org/account/register/).
+You also need to be a maintainer (or an owner) of the [apache-beam](https://pypi.python.org/pypi/apache-beam) package in order to push a new release.
+Ask on the mailing list for assistance.
+
+Generate a [PyPI APIToken](https://pypi.org/help/#apitoken) for use during the release.
+
+#### Login to DockerHub
+If you are a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam), run the following command manually.
+It will ask you to input your DockerHub ID and password if authorization info cannot be found from ~/.docker/config.json file.
+
+```
+docker login docker.io
+```
+
+After successful login, authorization info will be stored at ~/.docker/config.json file.
+For example,
+```
+"https://index.docker.io/v1/": {
+ "auth": "xxxxxx"
+}
+```
+
+If you are not already a member of the `beam` team, please email `dev@` mailing list for help with any DockerHub related tasks. We are not able
+to add more members to the DockerHub team because [the ASF has a limited number of seats available](https://infra.apache.org/docker-hub-policy.html).
#### GPG Key
@@ -141,6 +194,14 @@ __NOTE__: When generating the key, please make sure you choose the key type as _
Here, the key ID is the 16-digit hex string in the `pub` line: `845E6689845E6689`.
+##### Submit your GPG public key into Ubuntu OpenPGP Key Server
+In order to make yourself have right permission to stage java artifacts in Apache Nexus staging repository,
+please submit your GPG public key into the [Ubuntu OpenPGP Key Server](https://keyserver.ubuntu.com/).
+
+You will need to use an ascii-armored version of your key.
+This can be obtained by running `gpg --export --armor` and copying the whole block
+(including `----- PGP PUBLIC KEY BLOCK-----`).
+
#### Access to Apache Nexus repository
Configure access to the [Apache Nexus repository](https://repository.apache.org/), which enables final deployment of releases to the Maven Central Repository.
@@ -167,79 +228,34 @@ Configure access to the [Apache Nexus repository](https://repository.apache.org/
-#### Submit your GPG public key into Ubuntu OpenPGP Key Server
-In order to make yourself have right permission to stage java artifacts in Apache Nexus staging repository,
-please submit your GPG public key into the [Ubuntu OpenPGP Key Server](https://keyserver.ubuntu.com/).
-
-You will need to use an ascii-armored version of your key.
-This can be obtained by running `gpg --export --armor` and copying the whole block
-(including `----- PGP PUBLIC KEY BLOCK-----`).
-
-#### Website development setup
-
-Updating the Beam website requires submitting PRs to both the main `apache/beam` repo and the `apache/beam-site` repo.
-The first contains reference manuals generated from SDK code, while the second updates the current release version number.
-
-You should already have setup a local clone of `apache/beam`.
-Setting up a clone of `apache/beam-site` is similar:
+**********
- $ git clone -b release-docs https://github.com/apache/beam-site.git
- $ cd beam-site
- $ git remote add git@github.com:/beam-site.git
- $ git fetch --all
- $ git checkout -b origin/release-docs
+### Handle Per Release tasks
-Further instructions on website development on `apache/beam` is [here](https://github.com/apache/beam/blob/master/website).
-Background information about how the website is updated can be found in [Beam-Site Automation Reliability](https://s.apache.org/beam-site-automation).
+#### Update base image dependencies for Python container images
-#### Register to PyPI
+Tracked in Github issue https://github.com/apache/beam/issues/27944
-Release manager needs to have an account with PyPI.
-If you need one, [register at PyPI](https://pypi.python.org/account/register/).
-You also need to be a maintainer (or an owner) of the [apache-beam](https://pypi.python.org/pypi/apache-beam) package in order to push a new release.
-Ask on the mailing list for assistance.
-
-#### Login to DockerHub
-If you are a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam), run following command manually.
-It will ask you to input your DockerHub ID and password if authorization info cannot be found from ~/.docker/config.json file.
-
-```
-docker login docker.io
-```
-
-After successful login, authorization info will be stored at ~/.docker/config.json file.
-For example,
-```
-"https://index.docker.io/v1/": {
- "auth": "xxxxxx"
-}
-```
-
-If you are not already a member of the `beam` team, please email `dev@` for help with any DockerHub related tasks. We are not able
-to add more members to the DockerHub team because [the ASF has a limited number of seats available](https://infra.apache.org/docker-hub-policy.html).
-
-### Create a new milestone in GitHub
-
-When contributors resolve an issue in GitHub, they are tagging it with a release that will contain their changes.
-With the release currently underway, new issues should be resolved against a subsequent future release.
-Therefore, you should create a release item for this subsequent release, as follows:
+See instructions at: https://s.apache.org/beam-python-requirements-generate
-1. In GitHub, navigate to [`Issues > Milestones > New Milestone`](https://github.com/apache/beam/milestones).
-1. Add a new release. Choose the next minor version number after the version currently underway, select the next release due date (generally 6 weeks from today’s date) as the `Start Date`, and choose `Create Milestone`.
-1. At the end of the release, go to the same page and mark the recently released version as closed.
+Ideally, do the update at least a week before the release cut, so that any issues
+related to the update have time to surface.
+#### Update Go version used for container builds
-**********
+Tracked in Github issue https://github.com/apache/beam/issues/27897
+Ideally, do the update at least a week before the release cut, so that any issues
+related to the update have time to surface.
-## 3. Update base image dependencies for Python container images
+#### Update the Java BOM
-See instructions at: https://s.apache.org/beam-python-requirements-generate
+Tracked in Github issue https://github.com/apache/beam/issues/28379
Ideally, do the update at least a week before the release cut, so that any issues
related to the update have time to surface.
-## 4. Investigate performance regressions
+#### Investigate performance regressions
Check the Beam load tests for possible performance regressions.
Measurements are available on [metrics.beam.apache.org](http://metrics.beam.apache.org).
@@ -254,14 +270,64 @@ If regressions are found, the release branch can still be created, but the regre
The role of the release manager is to file GitHub issues for each regression with the milestone set to the to-be-released version.
The release manager oversees these just like any other issue marked with the milestone of the release.
-The mailing list should be informed to allow fixing the regressions in the course of the release.
+The mailing list should be informed to allow fixing the regressions in the course of the release. Issues should be filed and tagged with the milestone.
+
+#### Triage release-blocking issues in GitHub
+
+There could be outstanding release-blocking issues, which should be triaged before proceeding to build a release candidate.
+We track them by assigning the blocked release to the issue's milestone before the issue is resolved.
+
+The release manager should triage what does and does not block a release.
+The list of release-blocking issues is available at the [milestone status page](https://github.com/apache/beam/milestones).
+Triage each unresolved issue with one of the following resolutions:
-## 5. Create a release branch in apache/beam repository
+* An issue should not block the release if the problem exists in the current released version or is a bug in new functionality that does not exist in the current released version.
+* An issue should be a blocker if the problem is a regression between the currently released version and the release in progress and has no easy workaround.
-Attention: Only committer has permission to create release branch in apache/beam.
+For all GitHub issues:
+
+* If the issue has been resolved and the GitHub issue was not updated, resolve it accordingly.
+
+For issues with type "Bug" or labeled "flaky":
+
+* If the issue is a known continuously failing test, it is not acceptable to defer this until the next release.
+ Please work with the Beam community to resolve the issue.
+* If the issue is a known flaky test, make an attempt to delegate a fix.
+ However, if the issue may take too long to fix (to the discretion of the release manager):
+ * Delegate manual testing of the flaky issue to ensure no release blocking issues.
+ * Update the milestone to the version of the next release.
+ Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate.
+
+For all other GitHub issues:
+
+* If the issue has not been resolved and it is acceptable to defer this until the next release, update the milestone to the new version you just created.
+ Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate.
+* If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed.
+ Instead, work with the Beam community to resolve the issue.
+
+If there is a bug found in the RC creation process/tools, those issues should be considered high priority and fixed in 7 days.
+
+### Checklist to proceed to the next step
+
+1. Next release has a milestone in github.
+1. You have your various account credentials prepared.
+1. Per Release tasks for the current release have been handled.
+1. Open issues/PRs against the current release have been notified.
+1. Performance Regressions have been investigated and had issues filed.
+1. It is the proposed branch cut day.
+
+**********
+
+## 3. Build a release candidate
+
+Building a release candidate involves creating a release branch, running validation tests against the branch, filing issues, cherry picking fixes,
+making a release candidate tag, and building all artifacts from that tag.
+
+### Create a release branch in apache/beam repository
+
+As a final step in preparation for the release, you should create the release branch, and update version information on the original branch.
+This should happen once per release. If additional release candidates are required, they are built from later versions of this branch.
-Release candidates are built from a release branch.
-As a final step in preparation for the release, you should create the release branch, push it to the Apache code repository, and update version information on the original branch.
The final state of the repository should match this diagram:
@@ -282,28 +348,27 @@ Some processes, including our archetype tests, rely on having a live SNAPSHOT of
Once the release branch is cut, these SNAPSHOT versions are no longer found, so builds will be broken until a new snapshot is available.
The workflow starts the nightly snapshot by creating an empty PR against apache:master (which will be linked to in the logs).
-#### Use cut_release_branch.sh to cut a release branch
+#### Use cut_release_branch workflow to cut a release branch
+
* **Action:** [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) (click `run workflow`)
-In order to run this workflow, you will need to provide a Jenkins username and API token. Your Jenkins username should be your Apache ID.
-Your Jenkins API token can be generated by visiting https://ci-beam.apache.org/user//configure and clicking
-`Add new token` in the API token section.
+In order to run this workflow, you will need to provide a Apache ID and Jenkins API token.
+Your Jenkins API token can be generated by visiting https://ci-beam.apache.org, signing in with your Apache credentials,
+then going to `https://ci-beam.apache.org/user//configure` and clicking `Add new token` in the API token section.
* Tasks you need to do manually to __verify the SNAPSHOT build__
1. Check whether the Jenkins job gets triggered. If not, please comment ```Run Gradle Publish``` into the generated PR.
1. After verifying build succeeded, you need to close PR manually.
1. Manually update `CHANGES.md` on `master` by adding a new section for the next release ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)).
-
-**********
-
-
-## 6. Verify release branch
+### Verify release branch
After the release branch is cut you need to make sure it builds and has no significant issues that would block the creation of the release candidate.
There are 2 ways to perform this verification, either running automation script(recommended), or running all commands manually.
-! Dataflow tests will fail if Dataflow worker container is not created and published by this time. (Should be done by Google)
+> Dataflow tests will fail if the Dataflow worker container is not created and published by this time.
+> Should be done by Google, in response to the creation of the release branch, and docker images are hosted.
+> This should not block creation of the first release candidate, but should block approval of the release.
#### Run automation script (verify_release_build.sh)
* **Script:** [verify_release_build.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/verify_release_build.sh)
@@ -312,7 +377,7 @@ There are 2 ways to perform this verification, either running automation script(
1. Create a personal access token from your Github account.
See instruction [here](https://help.github.com/en/articles/creating-a-personal-access-token-for-the-command-line).
It'll be used by the script for accessing Github API.
- You only need to enable "repo" permissions to this token.
+ You need to enable `repo` and `workflow` permissions for this token.
1. Update required configurations listed in `RELEASE_BUILD_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config)
1. Then run
```
@@ -325,7 +390,7 @@ There are 2 ways to perform this verification, either running automation script(
for a full list of phrases.
* **Tasks included in the script**
- 1. Installs ```hub``` with your agreement and setup local git repo;
+ 1. Installs `hub` with your agreement and setup local git repo;
1. Create a test PR against release branch;
#### Verify the build succeeds
@@ -339,7 +404,7 @@ There are some projects that don't produce the artifacts, e.g. `beam-test-tools`
To triage the failures and narrow things down you may want to look at `settings.gradle.kts` and run the build only for the projects you're interested at the moment, e.g. `./gradlew :runners:java-fn-execution`.
-#### (Alternative) Run release build manually (locally)
+#### (Alternative) Run release build locally
You will need to have Python interpreters for all supported Python minor
versions to run Python tests. See Python installation tips in [Developer Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters).
@@ -366,7 +431,6 @@ versions to run Python tests. See Python installation tips in [Developer Wiki](h
if you want build fails after the first error instead of continuing, it may be easier and faster
to find environment issues this way without having to wait until the full build completes.
-
#### Create release-blocking issues in GitHub
The verify_release_build.sh script may include failing or flaky tests.
@@ -386,55 +450,18 @@ For each of the failing tests create a GitHub Issue with the following propertie
#### Inform the mailing list
-The dev@beam.apache.org mailing list should be informed about the release branch being cut.
+The dev@ mailing list should be informed about the release branch being cut.
Alongside with this note, a list of pending issues and to-be-triaged issues should be included.
Afterwards, this list can be refined and updated by the release manager and the Beam community.
-**********
-
-
-## 7. Triage release-blocking issues in GitHub
-
-There could be outstanding release-blocking issues, which should be triaged before proceeding to build a release candidate.
-We track them by assigning the blocked release to the issue's milestone before the issue is resolved.
-
-
-The release manager should triage what does and does not block a release.
-The list of release-blocking issues is available at the [milestone status page](https://github.com/apache/beam/milestones).
-Triage each unresolved issue with one of the following resolutions:
-
-* An issue should not block the release if the problem exists in the current released version or is a bug in new functionality that does not exist in the current released version.
-* An issue should be a blocker if the problem is a regression between the currently released version and the release in progress and has no easy workaround.
-
-For all GitHub issues:
-
-* If the issue has been resolved and the GitHub issue was not updated, resolve it accordingly.
-
-For issues with type "Bug" or labeled "flaky":
-
-* If the issue is a known continuously failing test, it is not acceptable to defer this until the next release.
- Please work with the Beam community to resolve the issue.
-* If the issue is a known flaky test, make an attempt to delegate a fix.
- However, if the issue may take too long to fix (to the discretion of the release manager):
- * Delegate manual testing of the flaky issue to ensure no release blocking issues.
- * Update the milestone to the version of the next release.
- Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate.
-
-For all other GitHub issues:
-
-* If the issue has not been resolved and it is acceptable to defer this until the next release, update the milestone to the new version you just created.
- Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate.
-* If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed.
- Instead, work with the Beam community to resolve the issue.
-
-If there is a bug found in the RC creation process/tools, those issues should be considered high priority and fixed in 7 days.
-
### Review cherry-picks
+The release manager is empowered to triage issues, and accept or reject cherry-picks to the release branch.
+Cherry picks are necessary if there are outstanding issues at time of the release branch cut, or issues were found in verification.
+
Check if there are outstanding cherry-picks into the release branch, [e.g. for `2.14.0`](https://github.com/apache/beam/pulls?utf8=%E2%9C%93&q=is%3Apr+base%3Arelease-2.14.0).
Make sure they have blocker Issues attached and are OK to get into the release by checking with community if needed.
-As the Release Manager you are empowered to accept or reject cherry-picks to the release branch.
You are encouraged to ask the following questions to be answered on each cherry-pick PR and you can choose to reject cherry-pick requests if these questions are not satisfactorily answered:
* Is this a regression from a previous release? (If no, fix could go to a newer version.)
@@ -450,11 +477,9 @@ Neither late releases nor not fully tested code will provide positive user value
__Tip__: Another tool in your toolbox is the known issues section of the release blog.
Consider adding known issues there for minor issues instead of accepting cherry picks to the release branch.
+### Build release artifacts
-**********
-
-
-## 8. Build a release candidate
+Once the branch is verified, it's time to build
#### Checklist before proceeding
@@ -479,7 +504,7 @@ The Release Manager repeats this cycle until the community approves one release
For this step, we recommend you using automation script to create a RC, but you still can perform all steps manually if you want.
-### Tag a chosen commit for the RC
+#### Tag a chosen commit for the RC
Release candidates are built from single commits off the release branch.
Before building, the version must be set to a non-SNAPSHOT, non-dev version.
@@ -509,7 +534,7 @@ is perfectly safe since the script does not depend on the current working tree.
See the source of the script for more details, or to run commands manually in case of a problem.
-### Run build_release_candidate GitHub Action to create a release candidate
+#### Run build_release_candidate GitHub Action to create a release candidate
Note: This step is partially automated (in progress), so part of the rc creation is done by GitHub Actions and the rest is done by a script.
You don't need to wait for the action to complete to start running the script.
@@ -523,7 +548,7 @@ You don't need to wait for the action to complete to start running the script.
1. Build javadoc, pydoc, typedocs for a PR to update beam-site.
* **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release").
-#### Tasks you need to do manually
+##### Tasks you need to do manually
1. Publish staging artifacts
1. Log in to the [Apache Nexus](https://repository.apache.org/#stagingRepositories) website.
@@ -537,7 +562,7 @@ You don't need to wait for the action to complete to start running the script.
Carefully review any new artifacts.
Some additional validation should be done during the rc validation step.
-### Run build_release_candidate.sh to create a release candidate
+#### Run build_release_candidate.sh to create a release candidate
* **Script:** [build_release_candidate.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/build_release_candidate.sh)
@@ -553,14 +578,14 @@ Skip this step if you already did it with the build_release_candidate GitHub Act
1. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image).
Skip this step if you already did it with the build_release_candidate GitHub Actions workflow.
Note: if you are not a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam) you will need
-help with this step. Please email `dev@` and ask a member of the `beam` DockerHub team for help.
+help with this step. Please email `dev@` mailing list and ask a member of the `beam` DockerHub team for help.
1. Create a PR to update beam-site, changes includes:
* Copy python doc into beam-site
* Copy java doc into beam-site
* **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release").
Skip this step if you already did it with the build_release_candidate GitHub Actions workflow.
-#### Tasks you need to do manually
+##### Tasks you need to do manually
1. Verify the script worked.
1. Verify that the source and Python binaries are present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam).
1. Verify Docker images are published. How to find images:
@@ -581,7 +606,7 @@ Skip this step if you already did it with the build_release_candidate GitHub Act
ls -al /opt/apache/beam/third_party_licenses/ | wc -l
```
-### Upload release candidate to PyPi
+#### Upload release candidate to PyPi
* **Script:** [deploy_release_candidate_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_release_candidate_pypi.sh)
@@ -608,13 +633,6 @@ You can do a dry run by omitting the `--deploy` flag. Then it will only download
See the source of the script for more details or to run commands manually in case of a problem.
-
-
-**********
-
-
-## 9. Prepare documents
-
### Propose pull requests for website updates
Beam publishes API reference manuals for each release on the website.
@@ -629,7 +647,7 @@ A committer can manually trigger the [beam_PostCommit_Website_Publish](https://c
**PR 1: apache/beam-site**
This pull request is against the `apache/beam-site` repo, on the `release-docs` branch ([example](https://github.com/apache/beam-site/pull/603)).
-It is created by `build_release_candidate.sh` (see above).
+It is created by the `build_release_candidate` workflow (see above).
**PR 2: apache/beam**
@@ -728,7 +746,7 @@ all major features and bug fixes, and all known issues.
${CONTRIBUTORS}
-#### Checklist to proceed to the next step
+### Checklist to proceed to the next step
1. Maven artifacts deployed to the staging repository of [repository.apache.org](https://repository.apache.org/content/repositories/)
1. Source distribution deployed to the dev repository of [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/)
@@ -747,11 +765,9 @@ docker pull {image_name}
docker pull apache/beam_python3.7_sdk:2.39.0rc1
```
-
**********
-
-## 10. Vote and validate release candidate
+## 4. Vote and validate release candidate
Once you have built and individually reviewed the release candidate, please share it for the community-wide review.
Please review foundation-wide [voting guidelines](https://www.apache.org/foundation/voting.html) for more information.
@@ -809,7 +825,7 @@ Here’s an email template; please adjust as you see fit.
If there are any issues found in the release candidate, reply on the vote thread to cancel the vote.
There’s no need to wait 72 hours.
-Proceed to the `Fix Issues` step below and address the problem.
+Proceed to the `Fix issues` step below and address the problem.
However, some issues don’t require cancellation.
For example, if an issue is found in the website pull request, just correct it on the spot and the vote can continue as-is.
@@ -819,7 +835,7 @@ Before accepting an RC, as a community we try to exercise most (if not all) of t
[spreadsheet](https://s.apache.org/beam-release-validation), and those are good validations for you to try out as release manager.
The goal of these tests is to validate that we're able to run basic pipelines from a variety of environments (not just our CI environment).
-Since there are a bunch of tests, we recommend you running some validations using an automation script.
+Since there are many tests, we recommend you running some validations using an automation script.
In case of script failure, you can still run all of them manually.
You may need to have Python interpreters for all supported Python minor
@@ -1078,7 +1094,7 @@ _Note_: -Prepourl and -Pver can be found in the RC vote email sent by Release Ma
* bq head -n 10 ${USER}_test.game_stats_sessions
-### Fix any issues
+### Fix issues
Any issues identified during the community review and vote should be fixed in this step.
Additionally, any GitHub issues created from the initial branch verification should be fixed.
@@ -1086,7 +1102,7 @@ Additionally, any GitHub issues created from the initial branch verification sho
Code changes should be proposed as standard pull requests to the `master` branch and reviewed using the normal contributing process.
Then, relevant changes should be cherry-picked into the release branch proposed as pull requests against the release branch, again reviewed and merged using the normal contributing process.
-Once all issues have been resolved, you should go back and build a new release candidate with these changes.
+Once all issues have been resolved as in the `Verify release branch` step, you should go back and build a new release candidate with these changes.
### Finalize the vote
@@ -1123,12 +1139,13 @@ Here’s an email template; please adjust as you see fit.
**********
-
-## 11. Finalize the release
+## 5. Finalize the release
Once the release candidate has been reviewed and approved by the community, the release should be finalized.
This involves the final deployment of the release candidate to the release repositories, merging of the website changes, etc.
+This does not take very long, and can be accomplished within hours of the vote being finalized.
+
### Deploy artifacts to Maven Central Repository
Use the [Apache Nexus repository manager](https://repository.apache.org/#stagingRepositories) to release the staged binary artifacts to the Maven Central repository.
@@ -1152,7 +1169,7 @@ All wheels should be published, in addition to the zip of the release source.
### Deploy docker images to DockerHub
Note: if you are not a member of the [beam DockerHub team](https://hub.docker.com/orgs/apache/teams/beam),
-you will need help with this step. Please email dev@ and ask a member of the beam DockerHub team for help.
+you will need help with this step. Please email dev@ mailing list and ask a member of the beam DockerHub team for help.
* **Script:** [publish_docker_images.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/publish_docker_images.sh)
* **Usage**
@@ -1214,7 +1231,7 @@ In GitHub, in the [milestone page](https://github.com/apache/beam/milestones), c
### PMC-Only Finalization
There are a few release finalization tasks that only PMC members have permissions to do.
-Ping [dev@](mailto:dev@beam.apache.org) for assistance if you need it.
+Ping [dev@](mailto:dev@beam.apache.org) mailing list for assistance if you need it.
#### Deploy source release to dist.apache.org
@@ -1241,8 +1258,7 @@ Use [reporter.apache.org](https://reporter.apache.org/addrelease.html?beam) to s
**********
-
-## 12. Promote the release
+## 6. Promote the release
Once the release has been finalized, the last step of the process is to promote the release within the project and beyond.
@@ -1273,9 +1289,11 @@ Also, update [the Wikipedia article on Apache Beam](https://en.wikipedia.org/wik
**********
-## 13. Update Beam Playground
+## Post Release Tasks
+
+### Update Beam Playground
-After new Beam Release is published, Beam Playgorund can be updated following the steps below:
+After new Beam Release is published, Beam Playground can be updated following the steps below:
1. Open the [Cloud Build triggers in apache-beam-testing](https://console.cloud.google.com/cloud-build/triggers?project=apache-beam-testing) GCP project.
1. Find the trigger "Deploy-Update-Playground-environment-stg":
@@ -1283,15 +1301,15 @@ After new Beam Release is published, Beam Playgorund can be updated following th
1. Change the value for _SDK_TAG variable (Advanced -> Substitution Variables) to the actual version of Beam SDK (e.g. 2.47.0)
1. Click the Save button. The settings window should close without any errors
1. Click the RUN button next to the trigger name
- 1. Set the value for the _CONTAINER_TAG variable in format DD-MM-vXX (DD - day, MM - month, XX - version, e.g., 20-12-v01)
+ 1. In the panel that opened, set the value for the _CONTAINER_TAG variable in format DD-MM-vXX (DD - day, MM - month, XX - version, e.g., 20-12-v01)
1. Click the Run Trigger button
1. Open the [Trigger History](https://console.cloud.google.com/cloud-build/builds?project=apache-beam-testing) and wait for the job completion. Ensure that the job completed successfully (Status field shows a green tick)
-1. Find the trigger "Playground-CD-stable-manual-stg":
+1. Find the trigger "Playground-CD-stable-manual-stg", it will be run twice, once with default variables, and once with some overridden:
1. Click the RUN button next to the trigger name
- 1. Click the Run Trigger button (with default varaible vaues)
+ 1. In the panel that opened, click the Run Trigger button (with default variable values)
1. Open the [Trigger History](https://console.cloud.google.com/cloud-build/builds?project=apache-beam-testing) and wait for the job completion. Ensure that the job completed successfully (Status field shows a green tick)
1. Click the RUN button next to the trigger name
- 1. Change values for the variables:
+ 1. In the panel that opened, change values for the variables:
* _ORIGIN = PG_BEAMDOC
* _SUBDIRS = ./learning/beamdoc
1. Click the Run Trigger button
@@ -1312,7 +1330,7 @@ After new Beam Release is published, Beam Playgorund can be updated following th
1. Set the value for the _BEAM_VERSION variable (Advanced -> Substitution Variables) to the actual version of Beam SDK (e.g., 2.47.0)
1. Click the Save button. Click the Save button. The settings window should close without any errors
-## Improve the process
+### Improve the process
It is important that we improve the release processes over time.
Once you’ve finished the release, please take a step back and look what areas of this process and be improved. Perhaps some part of the process can be simplified.
diff --git a/website/www/site/content/en/documentation/ml/data-processing.md b/website/www/site/content/en/documentation/ml/data-processing.md
index 9b6ec5117a55..f6406353ea61 100755
--- a/website/www/site/content/en/documentation/ml/data-processing.md
+++ b/website/www/site/content/en/documentation/ml/data-processing.md
@@ -1,5 +1,5 @@
---
-title: "Data processing"
+title: "Data exploration"
---
-# Data processing
+# Data exploration
Several types of Apache Beam data processing are applicable to AI/ML projects:
- Data exploration: Learn about your data (properties, distributions, statistics) when you start to deploy your project or when the data changes.
diff --git a/website/www/site/content/en/documentation/ml/overview.md b/website/www/site/content/en/documentation/ml/overview.md
index fd2cf7185258..63aaf0f86e20 100644
--- a/website/www/site/content/en/documentation/ml/overview.md
+++ b/website/www/site/content/en/documentation/ml/overview.md
@@ -89,7 +89,8 @@ You can use Apache Beam for data validation and preprocessing by setting up data
{{< table >}}
| Task | Example |
| ------- | ---------------|
-| I want to explore my data | [Data Processing Example](/documentation/ml/data-processing) |:
+| I want to transform my data for preprocessing| [Preprocess data with MLTransform](/documentation/ml/preprocess-data) |
+| I want to explore my data | [Data exploration workflow and example](/documentation/ml/data-processing) |:
{{< /table >}}
diff --git a/website/www/site/content/en/documentation/ml/preprocess-data.md b/website/www/site/content/en/documentation/ml/preprocess-data.md
new file mode 100644
index 000000000000..cb79afff6036
--- /dev/null
+++ b/website/www/site/content/en/documentation/ml/preprocess-data.md
@@ -0,0 +1,229 @@
+---
+title: "Preprocess data"
+---
+
+
+# Preprocess data with MLTransform
+
+This page explains how to use the `MLTransform` class to preprocess data for machine learning (ML)
+workflows. Apache Beam provides a set of data processing transforms for
+preprocessing data for training and inference. The `MLTransform` class wraps the
+various transforms in one class, simplifying your workflow. For a full list of
+available transforms, see the [Transforms](#transforms) section on this page.
+
+The set of transforms currently available in the `MLTransform` class come from
+the TensorFlow Transforms (TFT) library. TFT offers specialized processing
+modules for machine learning tasks.
+
+## Why use MLTransform {#use-mltransform}
+
+- With `MLTransform`, you can use the same preprocessing steps for both
+ training and inference, which ensures consistent results.
+- Use `MLTransform` to transform a single example or a batch of
+ examples.
+- `MLTransform` can do a full pass on the dataset, which is useful when
+ you need to transform a single element only after analyzing the entire
+ dataset. For example, with `MLTransform`, you can complete the following tasks:
+ - Normalize an input value by using the minimum and maximum value
+ of the entire dataset.
+ - Convert `floats` to `ints` by assigning them buckets, based on
+ the observed data distribution.
+ - Convert `strings` to `ints` by generating vocabulary over the
+ entire dataset.
+ - Count the occurrences of words in all the documents to calculate
+ [TF-IDF](https://en.wikipedia.org/wiki/Tf%E2%80%93idf)
+ weights.
+
+## Support and limitations {#support}
+
+- Available in the Apache Beam Python SDK versions 2.50.0 and later.
+- Supports Python 3.8 and 3.9.
+- Only available for pipelines that use [default windows](/documentation/programming-guide/#single-global-window).
+- Only supports one-to-one transform mapping on a single element.
+
+## Transforms {#transforms}
+
+You can use `MLTransform` to perform the following data processing transforms.
+For information about the transforms, see
+[Module:tft](https://www.tensorflow.org/tfx/transform/api_docs/python/tft) in the
+TensorFlow documentation.
+
+{{< table >}}
+| Transform name | Description |
+| ------- | ---------------|
+| ApplyBuckets | See [`tft.apply_buckets`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/apply_buckets) in the TensorFlow documentation. |
+| Bucketize | See [`tft.bucketize`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/bucketize) in the TensorFlow documentation. |
+| ComputeAndApplyVocabulary | See [`tft.compute_and_apply_vocabulary`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/compute_and_apply_vocabulary) in the TensorFlow documentation. |
+| NGrams | See [`tft.ngrams`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/ngrams) in the TensorFlow documentation. |
+| ScaleByMinMax | See [`tft.scale_by_min_max`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_by_min_max) in the TensorFlow documentation. |
+| ScaleTo01 | See [`tft.scale_to_0_1`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_to_0_1) in the TensorFlow documentation. |
+| ScaleToZScore | See [`tft.scale_to_z_score`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_to_z_score) in the TensorFlow documentation. |
+| TFIDF | See [`tft.tfidf`](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/tfidf) in the TensorFlow documentation. |:
+{{< /table >}}
+
+Apply the transforms on either single or multiple columns passed as a
+`dict` on structured data. Keys are column names and values are lists containing
+each column's data.
+
+## I/O requirements {#io}
+
+- Input to the `MLTransform` class must be in one of the following formats:
+ - A `dict` of `str`
+ - Primitive types
+ - List of primitive types
+ - NumPy arrays
+- `MLTransform` outputs a Beam `Row` object with NumPy arrays.
+- The output `PCollection` is a schema `PCollection`. The output schema
+ contains the transformed columns.
+
+## Artifacts {#artifacts}
+
+Artifacts are additional data elements created by data transformations.
+Examples of artifacts are the minimum and maximum values from a `ScaleTo01`
+transformation, or the mean and variance from a `ScaleToZScore`
+transformation.
+
+In the `MLTransform` class, the `write_artifact_location` and the
+`read_artifact_location` parameters determine
+whether the `MLTransform` class creates artifacts or retrieves
+artifacts.
+
+### Write mode {#write-mode}
+
+When you use the `write_artifact_location` parameter, the `MLTransform` class runs the
+specified transformations on the dataset and then creates artifacts from these
+transformations. The artifacts are stored in the location that you specify in
+the `write_artifact_location` parameter or in the `MLTransform` output.
+
+Write mode is useful when you want to store the results of your transformations
+for future use. For example, if you apply the same transformations on a
+different dataset, use write mode to ensure that the transformation parameters
+remain consistent.
+
+The following examples demonstrate how write mode works.
+
+- The `ComputeAndApplyVocabulary` transform generates a vocabulary file that contains the
+ vocabulary generated over the entire dataset. The vocabulary file is stored in
+ the location specified by the `write_artifact_location` parameter value.
+ The `ComputeAndApplyVocabulary`
+ transform outputs the indices of the vocabulary to the vocabulary file.
+- The `ScaleToZScore` transform calculates the mean and variance over the entire dataset
+ and then normalizes the entire dataset using the mean and variance. The
+ mean and variance are outputted by the `MLTransform` operation.
+ When you use the `write_artifact_location` parameter, these
+ values are stored as a `tensorflow` graph in the location specified by
+ the `write_artifact_location` parameter value. You can reuse the values in read mode
+ to ensure that future transformations use the same mean and variance for normalization.
+
+### Read mode {#read-mode}
+
+When you use the `read_artifact_location` parameter, the `MLTransform` class expects the
+artifacts to exist in the value provided in the `read_artifact_location` parameter.
+In this mode, `MLTransform` retrieves the artifacts and uses them in the
+transform. Because the transformations are stored in the artifacts when you use
+read mode, you don't need to specify the transformations.
+
+### Artifact workflow {#artifact-workflow}
+
+The following scenario provides an example use case for artifacts.
+
+Before training a machine learning model, you use `MLTransform` with the
+`write_artifact_location` parameter.
+When you run `MLTransform`, it applies transformations that preprocess the
+dataset. The transformation produces artifacts that are stored in the location
+specified by the `write_artifact_location` parameter value.
+
+After preprocessing, you use the transformed data to train the machine learning
+model.
+
+After training, you run inference. You use new test data and use the
+`read_artifact_location` parameter. By using this setting, you ensure that the test
+data undergoes the same preprocessing steps as the training data. In read
+mode, running `MLTransform` fetches the transformation artifacts from the
+location specified in the `read_artifact_location` parameter value.
+`MLTransform` applies these artifacts to the test data.
+
+This workflow provides consistency in preprocessing steps for both training and
+test data. This consistency ensures that the model can accurately evaluate the
+test data and maintain the integrity of the model's performance.
+
+## Preprocess data with MLTransform {#use-mltransform}
+
+To use the `MLTransform` transform to preprocess data, add the following code to
+your pipeline:
+
+```
+ import apache_beam as beam
+ from apache_beam.ml.transforms.base import MLTransform
+ from apache_beam.ml.transforms.tft import
+ import tempfile
+
+ data = [
+ {
+