From ca744ae9f6511c58ee08c391bc06e63c0f90ecce Mon Sep 17 00:00:00 2001 From: martin trieu Date: Mon, 5 Aug 2024 16:04:28 +0700 Subject: [PATCH 01/21] Add WorkProvider interfaces and implementations (#31883) --- .../worker/StreamingDataflowWorker.java | 342 +++++++----------- .../FanOutStreamingEngineWorkerHarness.java} | 63 ++-- .../harness/SingleSourceWorkerHarness.java | 284 +++++++++++++++ .../StreamingEngineConnectionState.java | 2 +- .../harness/StreamingWorkerHarness.java | 28 ++ .../harness}/WindmillStreamSender.java | 20 +- .../windmill/client/grpc/ChannelzServlet.java | 27 +- .../grpc/GetWorkResponseChunkAssembler.java | 139 +++++++ .../client/grpc/GrpcDirectGetWorkStream.java | 148 +++----- .../client/grpc/GrpcDispatcherClient.java | 4 +- .../client/grpc/GrpcGetWorkStream.java | 131 ++----- .../windmill/work/WorkItemScheduler.java | 4 - .../budget/EvenGetWorkBudgetDistributor.java | 27 +- .../work/budget/GetWorkBudgetDistributor.java | 5 +- .../work/budget/GetWorkBudgetSpender.java | 32 ++ .../worker/StreamingDataflowWorkerTest.java | 2 +- ...nOutStreamingEngineWorkerHarnessTest.java} | 40 +- .../harness}/WindmillStreamSenderTest.java | 5 +- .../client/grpc/ChannelzServletTest.java | 6 +- .../client/grpc/GrpcWindmillServerTest.java | 12 +- .../EvenGetWorkBudgetDistributorTest.java | 126 +++---- 21 files changed, 839 insertions(+), 608 deletions(-) rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc/StreamingEngineClient.java => streaming/harness/FanOutStreamingEngineWorkerHarness.java} (91%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc => streaming/harness}/StreamingEngineConnectionState.java (97%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc => streaming/harness}/WindmillStreamSender.java (93%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc/StreamingEngineClientTest.java => streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java} (93%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{windmill/client/grpc => streaming/harness}/WindmillStreamSenderTest.java (97%) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index f196852b2253..90f072be997e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -23,14 +23,13 @@ import com.google.api.services.dataflow.model.CounterUpdate; import com.google.api.services.dataflow.model.MapTask; import com.google.auto.value.AutoValue; -import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -48,26 +47,25 @@ import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; import org.apache.beam.runners.dataflow.worker.streaming.ComputationStateCache; import org.apache.beam.runners.dataflow.worker.streaming.StageInfo; -import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; -import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.streaming.WorkHeartbeatResponseProcessor; import org.apache.beam.runners.dataflow.worker.streaming.config.ComputationConfig; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingApplianceComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEngineComputationConfigFetcher; import org.apache.beam.runners.dataflow.worker.streaming.config.StreamingEnginePipelineConfig; +import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness; +import org.apache.beam.runners.dataflow.worker.streaming.harness.SingleSourceWorkerHarness.GetWorkSender; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters; +import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerHarness; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerStatusPages; import org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingWorkerStatusReporter; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.appliance.JniWindmillApplianceServer; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream; -import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamPool; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.CompleteCommit; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.StreamingApplianceWorkCommitter; @@ -104,12 +102,11 @@ import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheStats; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -119,7 +116,8 @@ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) -public class StreamingDataflowWorker { +public final class StreamingDataflowWorker { + /** * Sinks are marked 'full' in {@link StreamingModeExecutionContext} once the amount of data sinked * (across all the sinks, if there are more than one) reaches this limit. This serves as hint for @@ -128,47 +126,44 @@ public class StreamingDataflowWorker { */ public static final int MAX_SINK_BYTES = 10_000_000; - // Maximum number of threads for processing. Currently, each thread processes one key at a time. - static final int MAX_PROCESSING_THREADS = 300; - static final long THREAD_EXPIRATION_TIME_SEC = 60; - static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; - static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class); - private static final Duration GET_DATA_STREAM_TIMEOUT = Duration.standardSeconds(30); + + /** + * Maximum number of threads for processing. Currently, each thread processes one key at a time. + */ + private static final int MAX_PROCESSING_THREADS = 300; /** The idGenerator to generate unique id globally. */ private static final IdGenerator ID_GENERATOR = IdGenerators.decrementingLongs(); - private static final int DEFAULT_STATUS_PORT = 8081; - // Maximum size of the result of a GetWork request. + /** Maximum size of the result of a GetWork request. */ private static final long MAX_GET_WORK_FETCH_BYTES = 64L << 20; // 64m /** Maximum number of failure stacktraces to report in each update sent to backend. */ private static final int MAX_FAILURES_TO_REPORT_IN_UPDATE = 1000; - private static final Random clientIdGenerator = new Random(); + private static final long THREAD_EXPIRATION_TIME_SEC = 60; + private static final Duration COMMIT_STREAM_TIMEOUT = Duration.standardMinutes(1); + private static final Duration GET_DATA_STREAM_TIMEOUT = Duration.standardSeconds(30); + private static final int DEFAULT_STATUS_PORT = 8081; + private static final Random CLIENT_ID_GENERATOR = new Random(); private static final String CHANNELZ_PATH = "/channelz"; - final WindmillStateCache stateCache; + + private final WindmillStateCache stateCache; private final StreamingWorkerStatusPages statusPages; private final ComputationConfig.Fetcher configFetcher; private final ComputationStateCache computationStateCache; private final BoundedQueueExecutor workUnitExecutor; - private final WindmillServerStub windmillServer; - private final Thread dispatchThread; + private final StreamingWorkerHarness streamingWorkerHarness; private final AtomicBoolean running = new AtomicBoolean(); private final DataflowWorkerHarnessOptions options; - private final long clientId; - private final GetDataClient getDataClient; - private final MemoryMonitor memoryMonitor; - private final Thread memoryMonitorThread; + private final BackgroundMemoryMonitor memoryMonitor; private final ReaderCache readerCache; private final DataflowExecutionStateSampler sampler = DataflowExecutionStateSampler.instance(); private final ActiveWorkRefresher activeWorkRefresher; private final WorkCommitter workCommitter; private final StreamingWorkerStatusReporter workerStatusReporter; private final StreamingCounters streamingCounters; - private final StreamingWorkScheduler streamingWorkScheduler; - private final HeartbeatSender heartbeatSender; private StreamingDataflowWorker( WindmillServerStub windmillServer, @@ -226,39 +221,42 @@ private StreamingDataflowWorker( this.workUnitExecutor = workUnitExecutor; - memoryMonitorThread = new Thread(memoryMonitor); - memoryMonitorThread.setPriority(Thread.MIN_PRIORITY); - memoryMonitorThread.setName("MemoryMonitor"); - - dispatchThread = - new Thread( - () -> { - LOG.info("Dispatch starting"); - if (windmillServiceEnabled) { - streamingDispatchLoop(); - } else { - dispatchLoop(); - } - LOG.info("Dispatch done"); - }); - dispatchThread.setDaemon(true); - dispatchThread.setPriority(Thread.MIN_PRIORITY); - dispatchThread.setName("DispatchThread"); - this.clientId = clientId; - this.windmillServer = windmillServer; + this.workerStatusReporter = workerStatusReporter; + this.streamingCounters = streamingCounters; + this.memoryMonitor = BackgroundMemoryMonitor.create(memoryMonitor); + StreamingWorkScheduler streamingWorkScheduler = + StreamingWorkScheduler.create( + options, + clock, + readerCache, + mapTaskExecutorFactory, + workUnitExecutor, + stateCache::forComputation, + failureTracker, + workFailureProcessor, + streamingCounters, + hotKeyLogger, + sampler, + operationalLimits, + ID_GENERATOR, + stageInfoMap); ThrottlingGetDataMetricTracker getDataMetricTracker = new ThrottlingGetDataMetricTracker(memoryMonitor); - + WorkerStatusPages workerStatusPages = + WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor); + StreamingWorkerStatusPages.Builder statusPagesBuilder = StreamingWorkerStatusPages.builder(); int stuckCommitDurationMillis; + GetDataClient getDataClient; + HeartbeatSender heartbeatSender; if (windmillServiceEnabled) { WindmillStreamPool getDataStreamPool = WindmillStreamPool.create( Math.max(1, options.getWindmillGetDataStreamCount()), GET_DATA_STREAM_TIMEOUT, windmillServer::getDataStream); - this.getDataClient = new StreamPoolGetDataClient(getDataMetricTracker, getDataStreamPool); - this.heartbeatSender = + getDataClient = new StreamPoolGetDataClient(getDataMetricTracker, getDataStreamPool); + heartbeatSender = new StreamPoolHeartbeatSender( options.getUseSeparateWindmillHeartbeatStreams() ? WindmillStreamPool.create( @@ -266,9 +264,16 @@ private StreamingDataflowWorker( : getDataStreamPool); stuckCommitDurationMillis = options.getStuckCommitDurationMillis() > 0 ? options.getStuckCommitDurationMillis() : 0; + statusPagesBuilder + .setDebugCapture( + new DebugCapture.Manager(options, workerStatusPages.getDebugCapturePages())) + .setChannelzServlet( + new ChannelzServlet( + CHANNELZ_PATH, options, windmillServer::getWindmillServiceEndpoints)) + .setWindmillStreamFactory(windmillStreamFactory); } else { - this.getDataClient = new ApplianceGetDataClient(windmillServer, getDataMetricTracker); - this.heartbeatSender = new ApplianceHeartbeatSender(windmillServer::getData); + getDataClient = new ApplianceGetDataClient(windmillServer, getDataMetricTracker); + heartbeatSender = new ApplianceHeartbeatSender(windmillServer::getData); stuckCommitDurationMillis = 0; } @@ -282,49 +287,40 @@ private StreamingDataflowWorker( executorSupplier.apply("RefreshWork"), getDataMetricTracker::trackHeartbeats); - WorkerStatusPages workerStatusPages = - WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor); - StreamingWorkerStatusPages.Builder statusPagesBuilder = - StreamingWorkerStatusPages.builder() + this.statusPages = + statusPagesBuilder .setClock(clock) .setClientId(clientId) .setIsRunning(running) .setStatusPages(workerStatusPages) .setStateCache(stateCache) - .setComputationStateCache(computationStateCache) + .setComputationStateCache(this.computationStateCache) .setCurrentActiveCommitBytes(workCommitter::currentActiveCommitBytes) .setGetDataStatusProvider(getDataClient::printHtml) - .setWorkUnitExecutor(workUnitExecutor); + .setWorkUnitExecutor(workUnitExecutor) + .build(); - this.statusPages = - windmillServiceEnabled - ? statusPagesBuilder - .setDebugCapture( - new DebugCapture.Manager(options, workerStatusPages.getDebugCapturePages())) - .setChannelzServlet(new ChannelzServlet(CHANNELZ_PATH, options, windmillServer)) - .setWindmillStreamFactory(windmillStreamFactory) - .build() - : statusPagesBuilder.build(); + Windmill.GetWorkRequest request = + Windmill.GetWorkRequest.newBuilder() + .setClientId(clientId) + .setMaxItems(chooseMaximumBundlesOutstanding()) + .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) + .build(); - this.workerStatusReporter = workerStatusReporter; - this.streamingCounters = streamingCounters; - this.memoryMonitor = memoryMonitor; - this.streamingWorkScheduler = - StreamingWorkScheduler.create( - options, - clock, - readerCache, - mapTaskExecutorFactory, - workUnitExecutor, - stateCache::forComputation, - failureTracker, - workFailureProcessor, - streamingCounters, - hotKeyLogger, - sampler, - operationalLimits, - ID_GENERATOR, - stageInfoMap); + this.streamingWorkerHarness = + SingleSourceWorkerHarness.builder() + .setStreamingWorkScheduler(streamingWorkScheduler) + .setWorkCommitter(workCommitter) + .setGetDataClient(getDataClient) + .setComputationStateFetcher(this.computationStateCache::get) + .setWaitForResources(() -> memoryMonitor.waitForResources("GetWork")) + .setHeartbeatSender(heartbeatSender) + .setGetWorkSender( + windmillServiceEnabled + ? GetWorkSender.forStreamingEngine( + receiver -> windmillServer.getWorkStream(request, receiver)) + : GetWorkSender.forAppliance(() -> windmillServer.getWork(request))) + .build(); LOG.debug("windmillServiceEnabled: {}", windmillServiceEnabled); LOG.debug("WindmillServiceEndpoint: {}", options.getWindmillServiceEndpoint()); @@ -333,7 +329,7 @@ private StreamingDataflowWorker( } public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions options) { - long clientId = clientIdGenerator.nextLong(); + long clientId = CLIENT_ID_GENERATOR.nextLong(); MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options); ConcurrentMap stageInfo = new ConcurrentHashMap<>(); StreamingCounters streamingCounters = StreamingCounters.create(); @@ -438,9 +434,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o ComputationConfig.Fetcher configFetcher; WindmillServerStub windmillServer; ComputationStateCache computationStateCache; - GrpcDispatcherClient dispatcherClient = GrpcDispatcherClient.create(createStubFactory(options)); GrpcWindmillStreamFactory windmillStreamFactory; if (options.isEnableStreamingEngine()) { + GrpcDispatcherClient dispatcherClient = + GrpcDispatcherClient.create(createStubFactory(options)); configFetcher = StreamingEngineComputationConfigFetcher.create( options.getGlobalConfigRefreshPeriod().getMillis(), @@ -469,7 +466,10 @@ public static StreamingDataflowWorker fromOptions(DataflowWorkerHarnessOptions o options.getWindmillServiceStreamingRpcHealthCheckPeriodMs()) .build(); windmillServer = - GrpcWindmillServer.create(options, windmillStreamFactory, dispatcherClient); + GrpcWindmillServer.create( + options, + windmillStreamFactory, + GrpcDispatcherClient.create(createStubFactory(options))); } else { windmillStreamFactory = windmillStreamFactoryBuilder.build(); windmillServer = new JniWindmillApplianceServer(options.getLocalWindmillHostport()); @@ -704,10 +704,6 @@ private static ChannelCachingStubFactory createStubFactory( return ChannelCachingRemoteStubFactory.create(workerOptions.getGcpCredential(), channelCache); } - private static void sleep(int millis) { - Uninterruptibles.sleepUninterruptibly(millis, TimeUnit.MILLISECONDS); - } - private static int chooseMaxThreads(DataflowWorkerHarnessOptions options) { if (options.getNumberOfWorkerHarnessThreads() != 0) { return options.getNumberOfWorkerHarnessThreads(); @@ -736,7 +732,7 @@ private static void enableBigQueryMetrics() { } @VisibleForTesting - final void reportPeriodicWorkerUpdatesForTest() { + void reportPeriodicWorkerUpdatesForTest() { workerStatusReporter.reportPeriodicWorkerUpdates(); } @@ -765,6 +761,11 @@ int numCommitThreads() { return workCommitter.parallelism(); } + @VisibleForTesting + CacheStats getStateCacheStats() { + return stateCache.getCacheStats(); + } + @VisibleForTesting ComputationStateCache getComputationStateCache() { return computationStateCache; @@ -773,14 +774,10 @@ ComputationStateCache getComputationStateCache() { @SuppressWarnings("FutureReturnValueIgnored") public void start() { running.set(true); - configFetcher.start(); - - memoryMonitorThread.start(); - dispatchThread.start(); + memoryMonitor.start(); + streamingWorkerHarness.start(); sampler.start(); - - workCommitter.start(); workerStatusReporter.start(); activeWorkRefresher.start(); } @@ -794,121 +791,19 @@ private void startStatusPages() { void stop() { try { configFetcher.stop(); - activeWorkRefresher.stop(); statusPages.stop(); running.set(false); - dispatchThread.interrupt(); - dispatchThread.join(); - - workCommitter.stop(); - memoryMonitor.stop(); - memoryMonitorThread.join(); + streamingWorkerHarness.shutdown(); + memoryMonitor.shutdown(); workUnitExecutor.shutdown(); - computationStateCache.closeAndInvalidateAll(); - workerStatusReporter.stop(); } catch (Exception e) { LOG.warn("Exception while shutting down: ", e); } } - private void dispatchLoop() { - while (running.get()) { - memoryMonitor.waitForResources("GetWork"); - - int backoff = 1; - Windmill.GetWorkResponse workResponse = null; - do { - try { - workResponse = getWork(); - if (workResponse.getWorkCount() > 0) { - break; - } - } catch (WindmillServerStub.RpcException e) { - LOG.warn("GetWork failed, retrying:", e); - } - sleep(backoff); - backoff = Math.min(1000, backoff * 2); - } while (running.get()); - for (final Windmill.ComputationWorkItems computationWork : workResponse.getWorkList()) { - final String computationId = computationWork.getComputationId(); - Optional maybeComputationState = computationStateCache.get(computationId); - if (!maybeComputationState.isPresent()) { - continue; - } - - final ComputationState computationState = maybeComputationState.get(); - final Instant inputDataWatermark = - WindmillTimeUtils.windmillToHarnessWatermark(computationWork.getInputDataWatermark()); - Watermarks.Builder watermarks = - Watermarks.builder() - .setInputDataWatermark(Preconditions.checkNotNull(inputDataWatermark)) - .setSynchronizedProcessingTime( - WindmillTimeUtils.windmillToHarnessWatermark( - computationWork.getDependentRealtimeInputWatermark())); - - for (final Windmill.WorkItem workItem : computationWork.getWorkList()) { - streamingWorkScheduler.scheduleWork( - computationState, - workItem, - watermarks.setOutputDataWatermark(workItem.getOutputDataWatermark()).build(), - Work.createProcessingContext( - computationId, getDataClient, workCommitter::commit, heartbeatSender), - /* getWorkStreamLatencies= */ Collections.emptyList()); - } - } - } - } - - void streamingDispatchLoop() { - while (running.get()) { - GetWorkStream stream = - windmillServer.getWorkStream( - Windmill.GetWorkRequest.newBuilder() - .setClientId(clientId) - .setMaxItems(chooseMaximumBundlesOutstanding()) - .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) - .build(), - (String computation, - Instant inputDataWatermark, - Instant synchronizedProcessingTime, - Windmill.WorkItem workItem, - Collection getWorkStreamLatencies) -> - computationStateCache - .get(computation) - .ifPresent( - computationState -> { - memoryMonitor.waitForResources("GetWork"); - streamingWorkScheduler.scheduleWork( - computationState, - workItem, - Watermarks.builder() - .setInputDataWatermark(inputDataWatermark) - .setSynchronizedProcessingTime(synchronizedProcessingTime) - .setOutputDataWatermark(workItem.getOutputDataWatermark()) - .build(), - Work.createProcessingContext( - computationState.getComputationId(), - getDataClient, - workCommitter::commit, - heartbeatSender), - getWorkStreamLatencies); - })); - try { - // Reconnect every now and again to enable better load balancing. - // If at any point the server closes the stream, we will reconnect immediately; otherwise - // we half-close the stream after some time and create a new one. - if (!stream.awaitTermination(GET_WORK_STREAM_TIMEOUT_MINUTES, TimeUnit.MINUTES)) { - stream.halfClose(); - } - } catch (InterruptedException e) { - // Continue processing until !running.get() - } - } - } - private void onCompleteCommit(CompleteCommit completeCommit) { if (completeCommit.status() != Windmill.CommitStatus.OK) { readerCache.invalidateReader( @@ -927,15 +822,6 @@ private void onCompleteCommit(CompleteCommit completeCommit) { completeCommit.shardedKey(), completeCommit.workId())); } - private Windmill.GetWorkResponse getWork() { - return windmillServer.getWork( - Windmill.GetWorkRequest.newBuilder() - .setClientId(clientId) - .setMaxItems(chooseMaximumBundlesOutstanding()) - .setMaxBytes(MAX_GET_WORK_FETCH_BYTES) - .build()); - } - @VisibleForTesting public Iterable buildCounters() { return Iterables.concat( @@ -967,4 +853,34 @@ private static ConfigFetcherComputationStateCacheAndWindmillClient create( abstract GrpcWindmillStreamFactory windmillStreamFactory(); } + + /** + * Monitors memory pressure on a background executor. May be used to throttle calls, blocking if + * there is memory pressure. + */ + @AutoValue + abstract static class BackgroundMemoryMonitor { + private static BackgroundMemoryMonitor create(MemoryMonitor memoryMonitor) { + return new AutoValue_StreamingDataflowWorker_BackgroundMemoryMonitor( + memoryMonitor, + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("MemoryMonitor") + .setPriority(Thread.MIN_PRIORITY) + .build())); + } + + abstract MemoryMonitor memoryMonitor(); + + abstract ExecutorService executor(); + + private void start() { + executor().execute(memoryMonitor()); + } + + private void shutdown() { + memoryMonitor().stop(); + executor().shutdown(); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java similarity index 91% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java index b9573ff94cc9..3556b7ce2919 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet.toImmutableSet; @@ -47,6 +47,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.StreamGetDataClient; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.ThrottlingGetDataMetricTracker; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; @@ -68,16 +70,19 @@ import org.slf4j.LoggerFactory; /** - * Client for StreamingEngine. Given a {@link GetWorkBudget}, divides the budget and starts the - * {@link WindmillStream.GetWorkStream}(s). + * {@link StreamingWorkerHarness} implementation that manages fan out to multiple backend + * destinations. Given a {@link GetWorkBudget}, divides the budget and starts the {@link + * WindmillStream.GetWorkStream}(s). */ @Internal @CheckReturnValue @ThreadSafe -public final class StreamingEngineClient { - private static final Logger LOG = LoggerFactory.getLogger(StreamingEngineClient.class); +public final class FanOutStreamingEngineWorkerHarness implements StreamingWorkerHarness { + private static final Logger LOG = + LoggerFactory.getLogger(FanOutStreamingEngineWorkerHarness.class); private static final String PUBLISH_NEW_WORKER_METADATA_THREAD = "PublishNewWorkerMetadataThread"; private static final String CONSUME_NEW_WORKER_METADATA_THREAD = "ConsumeNewWorkerMetadataThread"; + private final JobHeader jobHeader; private final GrpcWindmillStreamFactory streamFactory; private final WorkItemScheduler workItemScheduler; @@ -101,7 +106,7 @@ public final class StreamingEngineClient { private volatile boolean started; @SuppressWarnings("FutureReturnValueIgnored") - private StreamingEngineClient( + private FanOutStreamingEngineWorkerHarness( JobHeader jobHeader, GetWorkBudget totalGetWorkBudget, GrpcWindmillStreamFactory streamFactory, @@ -152,23 +157,15 @@ private StreamingEngineClient( private static ExecutorService singleThreadedExecutorServiceOf(String threadName) { return Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder() - .setNameFormat(threadName) - .setUncaughtExceptionHandler( - (t, e) -> { - LOG.error( - "{} failed due to uncaught exception during execution. ", t.getName(), e); - throw new StreamingEngineClientException(e); - }) - .build()); + new ThreadFactoryBuilder().setNameFormat(threadName).build()); } /** - * Creates an instance of {@link StreamingEngineClient} in a non-started state. + * Creates an instance of {@link FanOutStreamingEngineWorkerHarness} in a non-started state. * * @implNote Does not block the calling thread. Callers must explicitly call {@link #start()}. */ - public static StreamingEngineClient create( + public static FanOutStreamingEngineWorkerHarness create( JobHeader jobHeader, GetWorkBudget totalGetWorkBudget, GrpcWindmillStreamFactory streamingEngineStreamFactory, @@ -178,7 +175,7 @@ public static StreamingEngineClient create( GrpcDispatcherClient dispatcherClient, Function workCommitterFactory, ThrottlingGetDataMetricTracker getDataMetricTracker) { - return new StreamingEngineClient( + return new FanOutStreamingEngineWorkerHarness( jobHeader, totalGetWorkBudget, streamingEngineStreamFactory, @@ -192,7 +189,7 @@ public static StreamingEngineClient create( } @VisibleForTesting - static StreamingEngineClient forTesting( + static FanOutStreamingEngineWorkerHarness forTesting( JobHeader jobHeader, GetWorkBudget totalGetWorkBudget, GrpcWindmillStreamFactory streamFactory, @@ -203,8 +200,8 @@ static StreamingEngineClient forTesting( long clientId, Function workCommitterFactory, ThrottlingGetDataMetricTracker getDataMetricTracker) { - StreamingEngineClient streamingEngineClient = - new StreamingEngineClient( + FanOutStreamingEngineWorkerHarness fanOutStreamingEngineWorkProvider = + new FanOutStreamingEngineWorkerHarness( jobHeader, totalGetWorkBudget, streamFactory, @@ -215,11 +212,12 @@ static StreamingEngineClient forTesting( clientId, workCommitterFactory, getDataMetricTracker); - streamingEngineClient.start(); - return streamingEngineClient; + fanOutStreamingEngineWorkProvider.start(); + return fanOutStreamingEngineWorkProvider; } @SuppressWarnings("ReturnValueIgnored") + @Override public synchronized void start() { Preconditions.checkState(!started, "StreamingEngineClient cannot start twice."); // Starts the stream, this value is memoized. @@ -270,7 +268,8 @@ private void startWorkerMetadataConsumer() { } @VisibleForTesting - public synchronized void finish() { + @Override + public synchronized void shutdown() { Preconditions.checkState(started, "StreamingEngineClient never started."); getWorkerMetadataStream.get().halfClose(); getWorkBudgetRefresher.stop(); @@ -334,10 +333,13 @@ private synchronized ImmutableMap createNewWindmil .collect( toImmutableMap( Function.identity(), - // Reuse existing stubs if they exist. endpoint -> - currentConnections.getOrDefault( - endpoint, WindmillConnection.from(endpoint, this::createWindmillStub)))); + // Reuse existing stubs if they exist. Optional.orElseGet only calls the + // supplier if the value is not present, preventing constructing expensive + // objects. + Optional.ofNullable(currentConnections.get(endpoint)) + .orElseGet( + () -> WindmillConnection.from(endpoint, this::createWindmillStub)))); } private synchronized ImmutableMap @@ -423,11 +425,4 @@ private CloudWindmillServiceV1Alpha1Stub createWindmillStub(Endpoint endpoint) { .map(channelCachingStubFactory::createWindmillServiceStub) .orElseGet(dispatcherClient::getWindmillServiceStub); } - - private static class StreamingEngineClientException extends IllegalStateException { - - private StreamingEngineClientException(Throwable exception) { - super(exception); - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java new file mode 100644 index 000000000000..bc93e6d89c41 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java @@ -0,0 +1,284 @@ +/* + * 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.streaming.harness; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; + +import com.google.auto.value.AutoBuilder; +import com.google.auto.value.AutoOneOf; +import java.util.Collections; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.streaming.ComputationState; +import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; +import org.apache.beam.runners.dataflow.worker.streaming.Work; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.RpcException; +import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; +import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; +import org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler; +import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; +import org.apache.beam.sdk.annotations.Internal; +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.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link StreamingWorkerHarness} implementations that fetch {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) from a single source. + */ +@Internal +public final class SingleSourceWorkerHarness implements StreamingWorkerHarness { + private static final Logger LOG = LoggerFactory.getLogger(SingleSourceWorkerHarness.class); + private static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3; + + private final AtomicBoolean isRunning; + private final WorkCommitter workCommitter; + private final GetDataClient getDataClient; + private final HeartbeatSender heartbeatSender; + private final StreamingWorkScheduler streamingWorkScheduler; + private final Runnable waitForResources; + private final Function> computationStateFetcher; + private final ExecutorService workProviderExecutor; + private final GetWorkSender getWorkSender; + + SingleSourceWorkerHarness( + WorkCommitter workCommitter, + GetDataClient getDataClient, + HeartbeatSender heartbeatSender, + StreamingWorkScheduler streamingWorkScheduler, + Runnable waitForResources, + Function> computationStateFetcher, + GetWorkSender getWorkSender) { + this.workCommitter = workCommitter; + this.getDataClient = getDataClient; + this.heartbeatSender = heartbeatSender; + this.streamingWorkScheduler = streamingWorkScheduler; + this.waitForResources = waitForResources; + this.computationStateFetcher = computationStateFetcher; + this.workProviderExecutor = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setPriority(Thread.MIN_PRIORITY) + .setNameFormat("DispatchThread") + .build()); + this.isRunning = new AtomicBoolean(false); + this.getWorkSender = getWorkSender; + } + + public static SingleSourceWorkerHarness.Builder builder() { + return new AutoBuilder_SingleSourceWorkerHarness_Builder(); + } + + @Override + public void start() { + Preconditions.checkState( + isRunning.compareAndSet(false, true), + "Multiple calls to {}.start() are not allowed.", + getClass()); + workCommitter.start(); + workProviderExecutor.execute( + () -> { + getDispatchLoop().run(); + LOG.info("Dispatch done"); + }); + } + + private Runnable getDispatchLoop() { + switch (getWorkSender.getKind()) { + case APPLIANCE: + LOG.info("Starting Dispatch in Appliance mode."); + return () -> applianceDispatchLoop(getWorkSender.appliance()); + case STREAMING_ENGINE: + LOG.info("Starting Dispatch in Streaming Engine mode."); + return () -> streamingEngineDispatchLoop(getWorkSender.streamingEngine()); + default: + // Will never happen switch is exhaustive. + throw new IllegalStateException("Invalid GetWorkSender.Kind: " + getWorkSender.getKind()); + } + } + + @Override + public void shutdown() { + Preconditions.checkState( + isRunning.compareAndSet(true, false), + "Multiple calls to {}.shutdown() are not allowed.", + getClass()); + workProviderExecutor.shutdown(); + boolean isTerminated = false; + try { + isTerminated = workProviderExecutor.awaitTermination(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn("Unable to shutdown {}", getClass()); + } + + if (!isTerminated) { + workProviderExecutor.shutdownNow(); + } + workCommitter.stop(); + } + + private void streamingEngineDispatchLoop( + Function getWorkStreamFactory) { + while (isRunning.get()) { + WindmillStream.GetWorkStream stream = + getWorkStreamFactory.apply( + (computationId, + inputDataWatermark, + synchronizedProcessingTime, + workItem, + getWorkStreamLatencies) -> + computationStateFetcher + .apply(computationId) + .ifPresent( + computationState -> { + waitForResources.run(); + streamingWorkScheduler.scheduleWork( + computationState, + workItem, + Watermarks.builder() + .setInputDataWatermark( + Preconditions.checkNotNull(inputDataWatermark)) + .setSynchronizedProcessingTime(synchronizedProcessingTime) + .setOutputDataWatermark(workItem.getOutputDataWatermark()) + .build(), + Work.createProcessingContext( + computationId, + getDataClient, + workCommitter::commit, + heartbeatSender), + getWorkStreamLatencies); + })); + try { + // Reconnect every now and again to enable better load balancing. + // If at any point the server closes the stream, we will reconnect immediately; otherwise + // we half-close the stream after some time and create a new one. + if (!stream.awaitTermination(GET_WORK_STREAM_TIMEOUT_MINUTES, TimeUnit.MINUTES)) { + stream.halfClose(); + } + } catch (InterruptedException e) { + // Continue processing until !running.get() + } + } + } + + private void applianceDispatchLoop(Supplier getWorkFn) { + while (isRunning.get()) { + waitForResources.run(); + int backoff = 1; + Windmill.GetWorkResponse workResponse = null; + do { + try { + workResponse = getWorkFn.get(); + if (workResponse.getWorkCount() > 0) { + break; + } + } catch (RpcException e) { + LOG.warn("GetWork failed, retrying:", e); + } + sleepUninterruptibly(backoff, TimeUnit.MILLISECONDS); + backoff = Math.min(1000, backoff * 2); + } while (isRunning.get()); + for (Windmill.ComputationWorkItems computationWork : + Preconditions.checkNotNull(workResponse).getWorkList()) { + String computationId = computationWork.getComputationId(); + Optional maybeComputationState = + computationStateFetcher.apply(computationId); + if (!maybeComputationState.isPresent()) { + continue; + } + + ComputationState computationState = maybeComputationState.get(); + Instant inputDataWatermark = + WindmillTimeUtils.windmillToHarnessWatermark(computationWork.getInputDataWatermark()); + Watermarks.Builder watermarks = + Watermarks.builder() + .setInputDataWatermark(Preconditions.checkNotNull(inputDataWatermark)) + .setSynchronizedProcessingTime( + WindmillTimeUtils.windmillToHarnessWatermark( + computationWork.getDependentRealtimeInputWatermark())); + + for (Windmill.WorkItem workItem : computationWork.getWorkList()) { + streamingWorkScheduler.scheduleWork( + computationState, + workItem, + watermarks.setOutputDataWatermark(workItem.getOutputDataWatermark()).build(), + Work.createProcessingContext( + computationId, getDataClient, workCommitter::commit, heartbeatSender), + /* getWorkStreamLatencies= */ Collections.emptyList()); + } + } + } + } + + @AutoBuilder + public interface Builder { + Builder setWorkCommitter(WorkCommitter workCommitter); + + Builder setGetDataClient(GetDataClient getDataClient); + + Builder setHeartbeatSender(HeartbeatSender heartbeatSender); + + Builder setStreamingWorkScheduler(StreamingWorkScheduler streamingWorkScheduler); + + Builder setWaitForResources(Runnable waitForResources); + + Builder setComputationStateFetcher( + Function> computationStateFetcher); + + Builder setGetWorkSender(GetWorkSender getWorkSender); + + SingleSourceWorkerHarness build(); + } + + @AutoOneOf(GetWorkSender.Kind.class) + public abstract static class GetWorkSender { + + public static GetWorkSender forStreamingEngine( + Function getWorkStreamFactory) { + return AutoOneOf_SingleSourceWorkerHarness_GetWorkSender.streamingEngine( + getWorkStreamFactory); + } + + public static GetWorkSender forAppliance(Supplier getWorkFn) { + return AutoOneOf_SingleSourceWorkerHarness_GetWorkSender.appliance(getWorkFn); + } + + abstract Function streamingEngine(); + + abstract Supplier appliance(); + + abstract Kind getKind(); + + enum Kind { + STREAMING_ENGINE, + APPLIANCE + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineConnectionState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingEngineConnectionState.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineConnectionState.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingEngineConnectionState.java index 8d784456d655..3c85ee6abe1f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineConnectionState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingEngineConnectionState.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import com.google.auto.value.AutoValue; import java.util.function.Supplier; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java new file mode 100644 index 000000000000..c1b4570e2260 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.dataflow.worker.streaming.harness; + +import org.apache.beam.sdk.annotations.Internal; + +/** Provides an interface to start streaming worker processing. */ +@Internal +public interface StreamingWorkerHarness { + void start(); + + void shutdown(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java similarity index 93% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java index 7d09726e4b28..45aa403ee71b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSender.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -29,9 +29,11 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.StreamingEngineThrottleTimers; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetSpender; import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FixedStreamHeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; @@ -57,7 +59,7 @@ */ @Internal @ThreadSafe -public class WindmillStreamSender { +final class WindmillStreamSender implements GetWorkBudgetSpender { private final AtomicBoolean started; private final AtomicReference getWorkBudget; private final Supplier getWorkStream; @@ -107,7 +109,7 @@ private WindmillStreamSender( workItemScheduler)); } - public static WindmillStreamSender create( + static WindmillStreamSender create( WindmillConnection connection, GetWorkRequest getWorkRequest, GetWorkBudget getWorkBudget, @@ -151,6 +153,7 @@ void closeAllStreams() { } } + @Override public void adjustBudget(long itemsDelta, long bytesDelta) { getWorkBudget.set(getWorkBudget.get().apply(itemsDelta, bytesDelta)); if (started.get()) { @@ -158,19 +161,16 @@ public void adjustBudget(long itemsDelta, long bytesDelta) { } } - public void adjustBudget(GetWorkBudget adjustment) { - adjustBudget(adjustment.items(), adjustment.bytes()); - } - - public GetWorkBudget remainingGetWorkBudget() { + @Override + public GetWorkBudget remainingBudget() { return started.get() ? getWorkStream.get().remainingBudget() : getWorkBudget.get(); } - public long getAndResetThrottleTime() { + long getAndResetThrottleTime() { return streamingEngineThrottleTimers.getAndResetThrottleTime(); } - public long getCurrentActiveCommitBytes() { + long getCurrentActiveCommitBytes() { return started.get() ? workCommitter.get().currentActiveCommitBytes() : 0; } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java index e0f823d79ade..adfb380d2164 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServlet.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.servlet.ServletException; @@ -31,7 +32,6 @@ import org.apache.beam.runners.dataflow.options.DataflowStreamingPipelineOptions; import org.apache.beam.runners.dataflow.worker.status.BaseStatusServlet; import org.apache.beam.runners.dataflow.worker.status.DebugCapture; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.channelz.v1.*; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.protobuf.services.ChannelzService; @@ -47,16 +47,16 @@ public class ChannelzServlet extends BaseStatusServlet implements DebugCapture.C private static final int MAX_TOP_CHANNELS_TO_RETURN = 500; private final ChannelzService channelzService; - private final WindmillServerStub windmillServerStub; + private final Supplier> currentWindmillEndpoints; private final boolean showOnlyWindmillServiceChannels; public ChannelzServlet( String path, DataflowStreamingPipelineOptions options, - WindmillServerStub windmillServerStub) { + Supplier> currentWindmillEndpoints) { super(path); channelzService = ChannelzService.newInstance(MAX_TOP_CHANNELS_TO_RETURN); - this.windmillServerStub = windmillServerStub; + this.currentWindmillEndpoints = currentWindmillEndpoints; showOnlyWindmillServiceChannels = options.getChannelzShowOnlyWindmillServiceChannels(); } @@ -81,14 +81,6 @@ public void captureData(PrintWriter writer) { writer.println(""); } - // channelz proto says there won't be cycles in the ref graph. - // we track visited ids to be defensive and prevent any accidental cycles. - private static class VisitedSets { - - Set channels = new HashSet<>(); - Set subchannels = new HashSet<>(); - } - private void appendTopChannels(PrintWriter writer) { SettableFuture future = SettableFuture.create(); // IDEA: If there are more than MAX_TOP_CHANNELS_TO_RETURN top channels @@ -127,8 +119,7 @@ private void appendTopChannels(PrintWriter writer) { } private List filterWindmillChannels(List channels) { - ImmutableSet windmillServiceEndpoints = - windmillServerStub.getWindmillServiceEndpoints(); + ImmutableSet windmillServiceEndpoints = currentWindmillEndpoints.get(); Set windmillServiceHosts = windmillServiceEndpoints.stream().map(HostAndPort::getHost).collect(Collectors.toSet()); List windmillChannels = new ArrayList<>(); @@ -291,4 +282,12 @@ public void onCompleted() { } }; } + + // channelz proto says there won't be cycles in the ref graph. + // we track visited ids to be defensive and prevent any accidental cycles. + private static class VisitedSets { + + Set channels = new HashSet<>(); + Set subchannels = new HashSet<>(); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java new file mode 100644 index 000000000000..9f30f75919f9 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java @@ -0,0 +1,139 @@ +/* + * 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.client.grpc; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import javax.annotation.Nullable; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link ByteString} buffer of {@link + * org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk}(s). + * + *

Once all serialized chunks of an {@link WorkItem} have been received flushes (deserializes) + * the chunk of bytes and metadata into an {@link AssembledWorkItem}. + * + * @implNote This class is not thread safe, and provides no synchronization underneath. + */ +@NotThreadSafe +final class GetWorkResponseChunkAssembler { + private static final Logger LOG = LoggerFactory.getLogger(GetWorkResponseChunkAssembler.class); + + private final GetWorkTimingInfosTracker workTimingInfosTracker; + private @Nullable ComputationMetadata metadata; + private ByteString data; + private long bufferedSize; + + GetWorkResponseChunkAssembler() { + workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis); + data = ByteString.EMPTY; + bufferedSize = 0; + metadata = null; + } + + /** + * Appends the response chunk bytes to the {@link #data }byte buffer. Return the assembled + * WorkItem if all response chunks for a WorkItem have been received. + */ + Optional append(Windmill.StreamingGetWorkResponseChunk chunk) { + if (chunk.hasComputationMetadata()) { + metadata = ComputationMetadata.fromProto(chunk.getComputationMetadata()); + } + + data = data.concat(chunk.getSerializedWorkItem()); + bufferedSize += chunk.getSerializedWorkItem().size(); + workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList()); + + // If the entire WorkItem has been received, assemble the WorkItem. + return chunk.getRemainingBytesForWorkItem() == 0 ? flushToWorkItem() : Optional.empty(); + } + + /** + * Attempt to flush the {@link #data} bytes into a {@link WorkItem} w/ it's metadata. Resets the + * data byte string and tracking metadata afterwards, whether the {@link WorkItem} deserialization + * was successful or not. + */ + private Optional flushToWorkItem() { + try { + return Optional.of( + AssembledWorkItem.create( + WorkItem.parseFrom(data.newInput()), + Preconditions.checkNotNull(metadata), + workTimingInfosTracker.getLatencyAttributions(), + bufferedSize)); + } catch (IOException e) { + LOG.error("Failed to parse work item from stream: ", e); + } finally { + workTimingInfosTracker.reset(); + data = ByteString.EMPTY; + bufferedSize = 0; + } + + return Optional.empty(); + } + + @AutoValue + abstract static class ComputationMetadata { + private static ComputationMetadata fromProto( + Windmill.ComputationWorkItemMetadata metadataProto) { + return new AutoValue_GetWorkResponseChunkAssembler_ComputationMetadata( + metadataProto.getComputationId(), + WindmillTimeUtils.windmillToHarnessWatermark(metadataProto.getInputDataWatermark()), + WindmillTimeUtils.windmillToHarnessWatermark( + metadataProto.getDependentRealtimeInputWatermark())); + } + + abstract String computationId(); + + abstract Instant inputDataWatermark(); + + abstract Instant synchronizedProcessingTime(); + } + + @AutoValue + abstract static class AssembledWorkItem { + + private static AssembledWorkItem create( + WorkItem workItem, + ComputationMetadata computationMetadata, + List latencyAttributions, + long size) { + return new AutoValue_GetWorkResponseChunkAssembler_AssembledWorkItem( + workItem, computationMetadata, latencyAttributions, size); + } + + abstract WorkItem workItem(); + + abstract ComputationMetadata computationMetadata(); + + abstract List latencyAttributions(); + + abstract long bufferedSize(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 58f72610e2d3..45d010d7cfac 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -17,8 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; -import com.google.auto.value.AutoValue; -import java.io.IOException; import java.io.PrintWriter; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -26,12 +24,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Supplier; -import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.ComputationWorkItemMetadata; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; @@ -40,6 +35,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GetWorkResponseChunkAssembler.AssembledWorkItem; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; @@ -47,13 +43,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; 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.Suppliers; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Implementation of {@link GetWorkStream} that passes along a specific {@link @@ -66,7 +58,6 @@ public final class GrpcDirectGetWorkStream extends AbstractWindmillStream implements GetWorkStream { - private static final Logger LOG = LoggerFactory.getLogger(GrpcDirectGetWorkStream.class); private static final StreamingGetWorkRequest HEALTH_CHECK_REQUEST = StreamingGetWorkRequest.newBuilder() .setRequestExtension( @@ -90,8 +81,10 @@ public final class GrpcDirectGetWorkStream * Map of stream IDs to their buffers. Used to aggregate streaming gRPC response chunks as they * come in. Once all chunks for a response has been received, the chunk is processed and the * buffer is cleared. + * + * @implNote Buffers are not persisted across stream restarts. */ - private final ConcurrentMap workItemBuffers; + private final ConcurrentMap workItemAssemblers; private GrpcDirectGetWorkStream( String backendWorkerToken, @@ -120,7 +113,7 @@ private GrpcDirectGetWorkStream( this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.workItemScheduler = workItemScheduler; - this.workItemBuffers = new ConcurrentHashMap<>(); + this.workItemAssemblers = new ConcurrentHashMap<>(); this.heartbeatSender = Suppliers.memoize(heartbeatSender::get); this.workCommitter = Suppliers.memoize(workCommitter::get); this.getDataClient = Suppliers.memoize(getDataClient::get); @@ -163,7 +156,8 @@ public static GrpcDirectGetWorkStream create( return getWorkStream; } - private static Watermarks createWatermarks(WorkItem workItem, ComputationMetadata metadata) { + private static Watermarks createWatermarks( + WorkItem workItem, GetWorkResponseChunkAssembler.ComputationMetadata metadata) { return Watermarks.builder() .setInputDataWatermark(metadata.inputDataWatermark()) .setOutputDataWatermark(workItem.getOutputDataWatermark()) @@ -171,14 +165,8 @@ private static Watermarks createWatermarks(WorkItem workItem, ComputationMetadat .build(); } - private synchronized GetWorkBudget getThenResetBudgetAdjustment() { - return nextBudgetAdjustment.getAndUpdate(unused -> GetWorkBudget.noBudget()); - } - - private void sendRequestExtension() { - // Just sent the request extension, reset the nextBudgetAdjustment. This will be set when - // adjustBudget is called. - GetWorkBudget adjustment = getThenResetBudgetAdjustment(); + private void sendRequestExtension(GetWorkBudget adjustment) { + inFlightBudget.getAndUpdate(budget -> budget.apply(adjustment)); StreamingGetWorkRequest extension = StreamingGetWorkRequest.newBuilder() .setRequestExtension( @@ -200,7 +188,7 @@ private void sendRequestExtension() { @Override protected synchronized void onNewStream() { - workItemBuffers.clear(); + workItemAssemblers.clear(); // Add the current in-flight budget to the next adjustment. Only positive values are allowed // here // with negatives defaulting to 0, since GetWorkBudgets cannot be created with negative values. @@ -229,7 +217,7 @@ public void appendSpecificHtml(PrintWriter writer) { // Number of buffers is same as distinct workers that sent work on this stream. writer.format( "GetWorkStream: %d buffers, %s inflight budget allowed.", - workItemBuffers.size(), inFlightBudget.get()); + workItemAssemblers.size(), inFlightBudget.get()); } @Override @@ -240,27 +228,49 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingGetWorkResponseChunk chunk) { getWorkThrottleTimer.stop(); - WorkItemBuffer workItemBuffer = - workItemBuffers.computeIfAbsent(chunk.getStreamId(), unused -> new WorkItemBuffer()); - workItemBuffer.append(chunk); + workItemAssemblers + .computeIfAbsent(chunk.getStreamId(), unused -> new GetWorkResponseChunkAssembler()) + .append(chunk) + .ifPresent(this::consumeAssembledWorkItem); + } - // The entire WorkItem has been received, it is ready to be processed. - if (chunk.getRemainingBytesForWorkItem() == 0) { - workItemBuffer.runAndReset(); - // Record the fact that there are now fewer outstanding messages and bytes on the stream. - inFlightBudget.updateAndGet(budget -> budget.subtract(1, workItemBuffer.bufferedSize())); + private void consumeAssembledWorkItem(AssembledWorkItem assembledWorkItem) { + // Record the fact that there are now fewer outstanding messages and bytes on the stream. + inFlightBudget.updateAndGet(budget -> budget.subtract(1, assembledWorkItem.bufferedSize())); + WorkItem workItem = assembledWorkItem.workItem(); + GetWorkResponseChunkAssembler.ComputationMetadata metadata = + assembledWorkItem.computationMetadata(); + pendingResponseBudget.getAndUpdate(budget -> budget.apply(1, workItem.getSerializedSize())); + try { + workItemScheduler.scheduleWork( + workItem, + createWatermarks(workItem, Preconditions.checkNotNull(metadata)), + createProcessingContext(Preconditions.checkNotNull(metadata.computationId())), + assembledWorkItem.latencyAttributions()); + } finally { + pendingResponseBudget.getAndUpdate(budget -> budget.apply(-1, -workItem.getSerializedSize())); } } + private Work.ProcessingContext createProcessingContext(String computationId) { + return Work.createProcessingContext( + computationId, getDataClient.get(), workCommitter.get()::commit, heartbeatSender.get()); + } + @Override protected void startThrottleTimer() { getWorkThrottleTimer.start(); } @Override - public synchronized void adjustBudget(long itemsDelta, long bytesDelta) { - nextBudgetAdjustment.set(nextBudgetAdjustment.get().apply(itemsDelta, bytesDelta)); - sendRequestExtension(); + public void adjustBudget(long itemsDelta, long bytesDelta) { + GetWorkBudget adjustment = + nextBudgetAdjustment + // Get the current value, and reset the nextBudgetAdjustment. This will be set again + // when adjustBudget is called. + .getAndUpdate(unused -> GetWorkBudget.noBudget()) + .apply(itemsDelta, bytesDelta); + sendRequestExtension(adjustment); } @Override @@ -274,74 +284,4 @@ public GetWorkBudget remainingBudget() { .apply(currentNextBudgetAdjustment) .apply(currentInflightBudget); } - - private synchronized void updatePendingResponseBudget(long itemsDelta, long bytesDelta) { - pendingResponseBudget.set(pendingResponseBudget.get().apply(itemsDelta, bytesDelta)); - } - - @AutoValue - abstract static class ComputationMetadata { - private static ComputationMetadata fromProto(ComputationWorkItemMetadata metadataProto) { - return new AutoValue_GrpcDirectGetWorkStream_ComputationMetadata( - metadataProto.getComputationId(), - WindmillTimeUtils.windmillToHarnessWatermark(metadataProto.getInputDataWatermark()), - WindmillTimeUtils.windmillToHarnessWatermark( - metadataProto.getDependentRealtimeInputWatermark())); - } - - abstract String computationId(); - - abstract Instant inputDataWatermark(); - - abstract Instant synchronizedProcessingTime(); - } - - private class WorkItemBuffer { - private final GetWorkTimingInfosTracker workTimingInfosTracker; - private ByteString data; - private @Nullable ComputationMetadata metadata; - - private WorkItemBuffer() { - workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis); - data = ByteString.EMPTY; - this.metadata = null; - } - - private void append(StreamingGetWorkResponseChunk chunk) { - if (chunk.hasComputationMetadata()) { - this.metadata = ComputationMetadata.fromProto(chunk.getComputationMetadata()); - } - - this.data = data.concat(chunk.getSerializedWorkItem()); - workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList()); - } - - private long bufferedSize() { - return data.size(); - } - - private void runAndReset() { - try { - WorkItem workItem = WorkItem.parseFrom(data.newInput()); - updatePendingResponseBudget(1, workItem.getSerializedSize()); - workItemScheduler.scheduleWork( - workItem, - createWatermarks(workItem, Preconditions.checkNotNull(metadata)), - createProcessingContext(Preconditions.checkNotNull(metadata.computationId())), - // After the work item is successfully queued or dropped by ActiveWorkState, remove it - // from the pendingResponseBudget. - queuedWorkItem -> updatePendingResponseBudget(-1, -workItem.getSerializedSize()), - workTimingInfosTracker.getLatencyAttributions()); - } catch (IOException e) { - LOG.error("Failed to parse work item from stream: ", e); - } - workTimingInfosTracker.reset(); - data = ByteString.EMPTY; - } - - private Work.ProcessingContext createProcessingContext(String computationId) { - return Work.createProcessingContext( - computationId, getDataClient.get(), workCommitter.get()::commit, heartbeatSender.get()); - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java index 033990017b24..cf2e7260592d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -75,7 +75,7 @@ public static GrpcDispatcherClient create(WindmillStubFactory windmillStubFactor } @VisibleForTesting - static GrpcDispatcherClient forTesting( + public static GrpcDispatcherClient forTesting( WindmillStubFactory windmillGrpcStubFactory, List windmillServiceStubs, List windmillMetadataServiceStubs, @@ -106,7 +106,7 @@ ImmutableSet getDispatcherEndpoints() { } /** Will block the calling thread until the initial endpoints are present. */ - CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStubBlocking() { + public CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStubBlocking() { boolean initialized = false; long secondsWaited = 0; while (!initialized) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index 4b392e9190ed..09ecbf3f3051 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -17,45 +17,34 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; -import java.io.IOException; import java.io.PrintWriter; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; 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.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.LatencyAttribution; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkRequestExtension; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.StreamingGetWorkResponseChunk; import org.apache.beam.runners.dataflow.worker.windmill.client.AbstractWindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GetWorkResponseChunkAssembler.AssembledWorkItem; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.sdk.util.BackOff; -import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public final class GrpcGetWorkStream +final class GrpcGetWorkStream extends AbstractWindmillStream implements GetWorkStream { - private static final Logger LOG = LoggerFactory.getLogger(GrpcGetWorkStream.class); - private final GetWorkRequest request; private final WorkItemReceiver receiver; private final ThrottleTimer getWorkThrottleTimer; - private final Map buffers; + private final Map workItemAssemblers; private final AtomicLong inflightMessages; private final AtomicLong inflightBytes; @@ -83,7 +72,7 @@ private GrpcGetWorkStream( this.request = request; this.getWorkThrottleTimer = getWorkThrottleTimer; this.receiver = receiver; - this.buffers = new ConcurrentHashMap<>(); + this.workItemAssemblers = new ConcurrentHashMap<>(); this.inflightMessages = new AtomicLong(); this.inflightBytes = new AtomicLong(); } @@ -138,7 +127,7 @@ private void sendRequestExtension(long moreItems, long moreBytes) { @Override protected synchronized void onNewStream() { - buffers.clear(); + workItemAssemblers.clear(); inflightMessages.set(request.getMaxItems()); inflightBytes.set(request.getMaxBytes()); send(StreamingGetWorkRequest.newBuilder().setRequest(request).build()); @@ -154,7 +143,7 @@ public void appendSpecificHtml(PrintWriter writer) { // Number of buffers is same as distinct workers that sent work on this stream. writer.format( "GetWorkStream: %d buffers, %d inflight messages allowed, %d inflight bytes allowed", - buffers.size(), inflightMessages.intValue(), inflightBytes.intValue()); + workItemAssemblers.size(), inflightMessages.intValue(), inflightBytes.intValue()); } @Override @@ -169,30 +158,33 @@ public void sendHealthCheck() { @Override protected void onResponse(StreamingGetWorkResponseChunk chunk) { getWorkThrottleTimer.stop(); + workItemAssemblers + .computeIfAbsent(chunk.getStreamId(), unused -> new GetWorkResponseChunkAssembler()) + .append(chunk) + .ifPresent(this::consumeAssembledWorkItem); + } - GrpcGetWorkStream.WorkItemBuffer buffer = - buffers.computeIfAbsent( - chunk.getStreamId(), unused -> new GrpcGetWorkStream.WorkItemBuffer()); - buffer.append(chunk); - - if (chunk.getRemainingBytesForWorkItem() == 0) { - long size = buffer.bufferedSize(); - buffer.runAndReset(); - - // Record the fact that there are now fewer outstanding messages and bytes on the stream. - long numInflight = inflightMessages.decrementAndGet(); - long bytesInflight = inflightBytes.addAndGet(-size); - - // If the outstanding items or bytes limit has gotten too low, top both off with a - // GetWorkExtension. The goal is to keep the limits relatively close to their maximum - // values without sending too many extension requests. - if (numInflight < request.getMaxItems() / 2 || bytesInflight < request.getMaxBytes() / 2) { - long moreItems = request.getMaxItems() - numInflight; - long moreBytes = request.getMaxBytes() - bytesInflight; - inflightMessages.getAndAdd(moreItems); - inflightBytes.getAndAdd(moreBytes); - sendRequestExtension(moreItems, moreBytes); - } + private void consumeAssembledWorkItem(AssembledWorkItem assembledWorkItem) { + receiver.receiveWork( + assembledWorkItem.computationMetadata().computationId(), + assembledWorkItem.computationMetadata().inputDataWatermark(), + assembledWorkItem.computationMetadata().synchronizedProcessingTime(), + assembledWorkItem.workItem(), + assembledWorkItem.latencyAttributions()); + + // Record the fact that there are now fewer outstanding messages and bytes on the stream. + long numInflight = inflightMessages.decrementAndGet(); + long bytesInflight = inflightBytes.addAndGet(-assembledWorkItem.bufferedSize()); + + // If the outstanding items or bytes limit has gotten too low, top both off with a + // GetWorkExtension. The goal is to keep the limits relatively close to their maximum + // values without sending too many extension requests. + if (numInflight < request.getMaxItems() / 2 || bytesInflight < request.getMaxBytes() / 2) { + long moreItems = request.getMaxItems() - numInflight; + long moreBytes = request.getMaxBytes() - bytesInflight; + inflightMessages.getAndAdd(moreItems); + inflightBytes.getAndAdd(moreBytes); + sendRequestExtension(moreItems, moreBytes); } } @@ -213,63 +205,4 @@ public GetWorkBudget remainingBudget() { .setItems(request.getMaxItems() - inflightMessages.get()) .build(); } - - private class WorkItemBuffer { - private final GetWorkTimingInfosTracker workTimingInfosTracker; - private String computation; - @Nullable private Instant inputDataWatermark; - @Nullable private Instant synchronizedProcessingTime; - private ByteString data; - private long bufferedSize; - - @SuppressWarnings("initialization.fields.uninitialized") - WorkItemBuffer() { - workTimingInfosTracker = new GetWorkTimingInfosTracker(System::currentTimeMillis); - data = ByteString.EMPTY; - bufferedSize = 0; - } - - @SuppressWarnings("NullableProblems") - private void setMetadata(Windmill.ComputationWorkItemMetadata metadata) { - this.computation = metadata.getComputationId(); - this.inputDataWatermark = - WindmillTimeUtils.windmillToHarnessWatermark(metadata.getInputDataWatermark()); - this.synchronizedProcessingTime = - WindmillTimeUtils.windmillToHarnessWatermark( - metadata.getDependentRealtimeInputWatermark()); - } - - private void append(StreamingGetWorkResponseChunk chunk) { - if (chunk.hasComputationMetadata()) { - setMetadata(chunk.getComputationMetadata()); - } - - this.data = data.concat(chunk.getSerializedWorkItem()); - this.bufferedSize += chunk.getSerializedWorkItem().size(); - workTimingInfosTracker.addTimingInfo(chunk.getPerWorkItemTimingInfosList()); - } - - private long bufferedSize() { - return bufferedSize; - } - - private void runAndReset() { - try { - Windmill.WorkItem workItem = Windmill.WorkItem.parseFrom(data.newInput()); - List getWorkStreamLatencies = - workTimingInfosTracker.getLatencyAttributions(); - receiver.receiveWork( - computation, - inputDataWatermark, - synchronizedProcessingTime, - workItem, - getWorkStreamLatencies); - } catch (IOException e) { - LOG.error("Failed to parse work item from stream: ", e); - } - workTimingInfosTracker.reset(); - data = ByteString.EMPTY; - bufferedSize = 0; - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java index 17c9f7d80d5d..00784493fe3d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/WorkItemScheduler.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.worker.windmill.work; import java.util.Collection; -import java.util.function.Consumer; import javax.annotation.CheckReturnValue; import org.apache.beam.runners.dataflow.worker.streaming.Watermarks; import org.apache.beam.runners.dataflow.worker.streaming.Work; @@ -36,8 +35,6 @@ public interface WorkItemScheduler { * @param workItem {@link WorkItem} to be processed. * @param watermarks processing watermarks for the workItem. * @param processingContext for processing the workItem. - * @param ackWorkItemQueued Called after an attempt to queue the work item for processing. Used to - * free up pending budget. * @param getWorkStreamLatencies Latencies per processing stage for the WorkItem for reporting * back to Streaming Engine backend. */ @@ -45,6 +42,5 @@ void scheduleWork( WorkItem workItem, Watermarks watermarks, Work.ProcessingContext processingContext, - Consumer ackWorkItemQueued, Collection getWorkStreamLatencies); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java index 3a17222d3e6b..403bb99efb4c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributor.java @@ -26,14 +26,13 @@ import java.util.Map.Entry; import java.util.function.Function; import java.util.function.Supplier; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** Evenly distributes the provided budget across the available {@link WindmillStreamSender}(s). */ +/** Evenly distributes the provided budget across the available {@link GetWorkBudgetSpender}(s). */ @Internal final class EvenGetWorkBudgetDistributor implements GetWorkBudgetDistributor { private static final Logger LOG = LoggerFactory.getLogger(EvenGetWorkBudgetDistributor.class); @@ -50,10 +49,10 @@ private static boolean isBelowFiftyPercentOfTarget( } @Override - public void distributeBudget( - ImmutableCollection streams, GetWorkBudget getWorkBudget) { - if (streams.isEmpty()) { - LOG.debug("Cannot distribute budget to no streams."); + public void distributeBudget( + ImmutableCollection budgetOwners, GetWorkBudget getWorkBudget) { + if (budgetOwners.isEmpty()) { + LOG.debug("Cannot distribute budget to no owners."); return; } @@ -62,23 +61,21 @@ public void distributeBudget( return; } - Map desiredBudgets = - computeDesiredBudgets(streams, getWorkBudget); + Map desiredBudgets = computeDesiredBudgets(budgetOwners, getWorkBudget); - for (Entry streamAndDesiredBudget : - desiredBudgets.entrySet()) { - WindmillStreamSender stream = streamAndDesiredBudget.getKey(); + for (Entry streamAndDesiredBudget : desiredBudgets.entrySet()) { + GetWorkBudgetSpender getWorkBudgetSpender = streamAndDesiredBudget.getKey(); GetWorkBudget desired = streamAndDesiredBudget.getValue(); - GetWorkBudget remaining = stream.remainingGetWorkBudget(); + GetWorkBudget remaining = getWorkBudgetSpender.remainingBudget(); if (isBelowFiftyPercentOfTarget(remaining, desired)) { GetWorkBudget adjustment = desired.subtract(remaining); - stream.adjustBudget(adjustment); + getWorkBudgetSpender.adjustBudget(adjustment); } } } - private ImmutableMap computeDesiredBudgets( - ImmutableCollection streams, GetWorkBudget totalGetWorkBudget) { + private ImmutableMap computeDesiredBudgets( + ImmutableCollection streams, GetWorkBudget totalGetWorkBudget) { GetWorkBudget activeWorkBudget = activeWorkBudgetSupplier.get(); LOG.info("Current active work budget: {}", activeWorkBudget); // TODO: Fix possibly non-deterministic handing out of budgets. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java index 3ec9718e041e..d21de17e522c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetDistributor.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.work.budget; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableCollection; @@ -28,6 +27,6 @@ */ @Internal public interface GetWorkBudgetDistributor { - void distributeBudget( - ImmutableCollection streams, GetWorkBudget getWorkBudget); + void distributeBudget( + ImmutableCollection streams, GetWorkBudget getWorkBudget); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java new file mode 100644 index 000000000000..254b2589062e --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/GetWorkBudgetSpender.java @@ -0,0 +1,32 @@ +/* + * 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.work.budget; + +/** + * Represents something that spends {@link + * org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget} + */ +public interface GetWorkBudgetSpender { + void adjustBudget(long itemsDelta, long bytesDelta); + + default void adjustBudget(GetWorkBudget adjustment) { + adjustBudget(adjustment.items(), adjustment.bytes()); + } + + GetWorkBudget remainingBudget(); +} 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 5855057c4210..d16ed2942fd9 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 @@ -2182,7 +2182,7 @@ public void testMergeWindowsCaching() throws Exception { // No input messages assertEquals(0L, splitIntToLong(getCounter(counters, "WindmillShuffleBytesRead").getInteger())); - CacheStats stats = worker.stateCache.getCacheStats(); + CacheStats stats = worker.getStateCacheStats(); LOG.info("cache stats {}", stats); assertEquals(1, stats.hitCount()); assertEquals(4, stats.missCount()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java similarity index 93% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java index 1999dbe31902..aaa71b6598ea 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarnessTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -49,12 +49,15 @@ import org.apache.beam.runners.dataflow.worker.windmill.WindmillServiceAddress; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.ThrottlingGetDataMetricTracker; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcDispatcherClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.ChannelCachingStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannelFactory; import org.apache.beam.runners.dataflow.worker.windmill.testing.FakeWindmillStubFactory; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetDistributor; +import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetSpender; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessSocketAddress; @@ -76,7 +79,7 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class StreamingEngineClientTest { +public class FanOutStreamingEngineWorkerHarnessTest { private static final WindmillServiceAddress DEFAULT_WINDMILL_SERVICE_ADDRESS = WindmillServiceAddress.create(HostAndPort.fromParts(WindmillChannelFactory.LOCALHOST, 443)); private static final ImmutableMap DEFAULT = @@ -113,14 +116,10 @@ public class StreamingEngineClientTest { private Server fakeStreamingEngineServer; private CountDownLatch getWorkerMetadataReady; private GetWorkerMetadataTestStub fakeGetWorkerMetadataStub; - private StreamingEngineClient streamingEngineClient; + private FanOutStreamingEngineWorkerHarness fanOutStreamingEngineWorkProvider; private static WorkItemScheduler noOpProcessWorkItemFn() { - return (workItem, - watermarks, - processingContext, - ackWorkItemQueued, - getWorkStreamLatencies) -> {}; + return (workItem, watermarks, processingContext, getWorkStreamLatencies) -> {}; } private static GetWorkRequest getWorkRequest(long items, long bytes) { @@ -163,16 +162,16 @@ public void setUp() throws IOException { @After public void cleanUp() { - Preconditions.checkNotNull(streamingEngineClient).finish(); + Preconditions.checkNotNull(fanOutStreamingEngineWorkProvider).shutdown(); fakeStreamingEngineServer.shutdownNow(); stubFactory.shutdown(); } - private StreamingEngineClient newStreamingEngineClient( + private FanOutStreamingEngineWorkerHarness newStreamingEngineClient( GetWorkBudget getWorkBudget, GetWorkBudgetDistributor getWorkBudgetDistributor, WorkItemScheduler workItemScheduler) { - return StreamingEngineClient.forTesting( + return FanOutStreamingEngineWorkerHarness.forTesting( JOB_HEADER, getWorkBudget, streamFactory, @@ -194,7 +193,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(numBudgetDistributionsExpected)); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(items).setBytes(bytes).build(), getWorkBudgetDistributor, @@ -216,7 +215,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { waitForWorkerMetadataToBeConsumed(getWorkBudgetDistributor); StreamingEngineConnectionState currentConnections = - streamingEngineClient.getCurrentConnections(); + fanOutStreamingEngineWorkProvider.getCurrentConnections(); assertEquals(2, currentConnections.windmillConnections().size()); assertEquals(2, currentConnections.windmillStreams().size()); @@ -250,7 +249,7 @@ public void testStreamsStartCorrectly() throws InterruptedException { public void testScheduledBudgetRefresh() throws InterruptedException { TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(2)); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(1L).setBytes(1L).build(), getWorkBudgetDistributor, @@ -273,7 +272,7 @@ public void testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers() int metadataCount = 2; TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(metadataCount)); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(1).setBytes(1).build(), getWorkBudgetDistributor, @@ -311,11 +310,12 @@ public void testOnNewWorkerMetadata_correctlyRemovesStaleWindmillServers() fakeGetWorkerMetadataStub.injectWorkerMetadata(secondWorkerMetadata); waitForWorkerMetadataToBeConsumed(getWorkBudgetDistributor); StreamingEngineConnectionState currentConnections = - streamingEngineClient.getCurrentConnections(); + fanOutStreamingEngineWorkProvider.getCurrentConnections(); assertEquals(1, currentConnections.windmillConnections().size()); assertEquals(1, currentConnections.windmillStreams().size()); Set workerTokens = - streamingEngineClient.getCurrentConnections().windmillConnections().values().stream() + fanOutStreamingEngineWorkProvider.getCurrentConnections().windmillConnections().values() + .stream() .map(WindmillConnection::backendWorkerToken) .collect(Collectors.toSet()); @@ -362,7 +362,7 @@ public void testOnNewWorkerMetadata_redistributesBudget() throws InterruptedExce TestGetWorkBudgetDistributor getWorkBudgetDistributor = spy(new TestGetWorkBudgetDistributor(workerMetadataResponses.size())); - streamingEngineClient = + fanOutStreamingEngineWorkProvider = newStreamingEngineClient( GetWorkBudget.builder().setItems(1).setBytes(1).build(), getWorkBudgetDistributor, @@ -439,8 +439,8 @@ private void waitForBudgetDistribution() throws InterruptedException { } @Override - public void distributeBudget( - ImmutableCollection streams, GetWorkBudget getWorkBudget) { + public void distributeBudget( + ImmutableCollection streams, GetWorkBudget getWorkBudget) { streams.forEach(stream -> stream.adjustBudget(getWorkBudget.items(), getWorkBudget.bytes())); getWorkBudgetDistributorTriggered.countDown(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java index 9d49c3ef3146..dc6cc5641055 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/WindmillStreamSenderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; +package org.apache.beam.runners.dataflow.worker.streaming.harness; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -35,6 +35,7 @@ import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; +import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; import org.apache.beam.runners.dataflow.worker.windmill.client.throttling.ThrottleTimer; import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemScheduler; import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget; @@ -64,7 +65,7 @@ public class WindmillStreamSenderTest { .build()) .build()); private final WorkItemScheduler workItemScheduler = - (workItem, watermarks, processingContext, ackWorkItemQueued, getWorkStreamLatencies) -> {}; + (workItem, watermarks, processingContext, getWorkStreamLatencies) -> {}; @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private ManagedChannel inProcessChannel; private WindmillConnection connection; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java index 96c675169a7d..d234cf424767 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/ChannelzServletTest.java @@ -56,7 +56,8 @@ public void testRendersAllChannels() throws UnsupportedEncodingException { fakeWindmillServer.setWindmillServiceEndpoints( ImmutableSet.of(HostAndPort.fromHost(windmill1), HostAndPort.fromHost(windmill2))); options.setChannelzShowOnlyWindmillServiceChannels(false); - ChannelzServlet channelzServlet = new ChannelzServlet("/channelz", options, fakeWindmillServer); + ChannelzServlet channelzServlet = + new ChannelzServlet("/channelz", options, fakeWindmillServer::getWindmillServiceEndpoints); StringWriter stringWriter = new StringWriter(); PrintWriter writer = new PrintWriter(stringWriter); channelzServlet.captureData(writer); @@ -88,7 +89,8 @@ public void testRendersOnlyWindmillChannels() throws UnsupportedEncodingExceptio fakeWindmillServer.setWindmillServiceEndpoints( ImmutableSet.of(HostAndPort.fromHost(windmill1), HostAndPort.fromHost(windmill2))); options.setChannelzShowOnlyWindmillServiceChannels(true); - ChannelzServlet channelzServlet = new ChannelzServlet("/channelz", options, fakeWindmillServer); + ChannelzServlet channelzServlet = + new ChannelzServlet("/channelz", options, fakeWindmillServer::getWindmillServiceEndpoints); StringWriter stringWriter = new StringWriter(); PrintWriter writer = new PrintWriter(stringWriter); channelzServlet.captureData(writer); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index 5cfc19ac07df..7e5801b65de4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -1142,7 +1142,13 @@ public void onNext(StreamingGetWorkRequest request) { StreamingGetWorkResponseChunk.newBuilder() .setStreamId(id) .setSerializedWorkItem(serializedResponse) - .setRemainingBytesForWorkItem(0); + .setRemainingBytesForWorkItem(0) + .setComputationMetadata( + ComputationWorkItemMetadata.newBuilder() + .setComputationId("computation") + .setInputDataWatermark(1L) + .setDependentRealtimeInputWatermark(1L) + .build()); try { responseObserver.onNext(builder.build()); } catch (IllegalStateException e) { @@ -1175,9 +1181,7 @@ public void onCompleted() { @Nullable Instant inputDataWatermark, Instant synchronizedProcessingTime, Windmill.WorkItem workItem, - Collection getWorkStreamLatencies) -> { - latch.countDown(); - }); + Collection getWorkStreamLatencies) -> latch.countDown()); // Wait for 100 items or 30 seconds. assertTrue(latch.await(30, TimeUnit.SECONDS)); // Confirm that we report at least as much throttle time as our server sent errors for. We will diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java index b0c305dc4ec4..3cda4559c100 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/EvenGetWorkBudgetDistributorTest.java @@ -19,7 +19,6 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -28,20 +27,8 @@ import java.util.ArrayList; import java.util.List; -import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; -import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection; -import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter; -import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.GrpcWindmillStreamFactory; -import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.WindmillStreamSender; -import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; -import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.junit.After; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; @@ -52,8 +39,6 @@ public class EvenGetWorkBudgetDistributorTest { @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); @Rule public transient Timeout globalTimeout = Timeout.seconds(600); - private ManagedChannel inProcessChannel; - private CloudWindmillServiceV1Alpha1Grpc.CloudWindmillServiceV1Alpha1Stub stub; private static GetWorkBudgetDistributor createBudgetDistributor(GetWorkBudget activeWorkBudget) { return GetWorkBudgetDistributors.distributeEvenly(() -> activeWorkBudget); @@ -67,20 +52,6 @@ private static GetWorkBudgetDistributor createBudgetDistributor(long activeWorkI .build()); } - @Before - public void setUp() { - inProcessChannel = - grpcCleanup.register( - InProcessChannelBuilder.forName("WindmillStreamSenderTest").directExecutor().build()); - grpcCleanup.register(inProcessChannel); - stub = CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel); - } - - @After - public void cleanUp() { - inProcessChannel.shutdownNow(); - } - @Test public void testDistributeBudget_doesNothingWhenPassedInStreamsEmpty() { createBudgetDistributor(1L) @@ -90,38 +61,40 @@ public void testDistributeBudget_doesNothingWhenPassedInStreamsEmpty() { @Test public void testDistributeBudget_doesNothingWithNoBudget() { - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(GetWorkBudget.noBudget())); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(GetWorkBudget.noBudget())); createBudgetDistributor(1L) - .distributeBudget(ImmutableList.of(windmillStreamSender), GetWorkBudget.noBudget()); - verifyNoInteractions(windmillStreamSender); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), GetWorkBudget.noBudget()); + verifyNoInteractions(getWorkBudgetSpender); } @Test public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHighNoActiveWork() { - WindmillStreamSender windmillStreamSender = + GetWorkBudgetSpender getWorkBudgetSpender = spy( - createWindmillStreamSender( + createGetWorkBudgetOwnerWithRemainingBudgetOf( GetWorkBudget.builder().setItems(10L).setBytes(10L).build())); createBudgetDistributor(0L) .distributeBudget( - ImmutableList.of(windmillStreamSender), + ImmutableList.of(getWorkBudgetSpender), GetWorkBudget.builder().setItems(10L).setBytes(10L).build()); - verify(windmillStreamSender, never()).adjustBudget(anyLong(), anyLong()); + verify(getWorkBudgetSpender, never()).adjustBudget(anyLong(), anyLong()); } @Test public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHighWithActiveWork() { - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(GetWorkBudget.builder().setItems(5L).setBytes(5L).build())); + GetWorkBudgetSpender getWorkBudgetSpender = + spy( + createGetWorkBudgetOwnerWithRemainingBudgetOf( + GetWorkBudget.builder().setItems(5L).setBytes(5L).build())); createBudgetDistributor(10L) .distributeBudget( - ImmutableList.of(windmillStreamSender), + ImmutableList.of(getWorkBudgetSpender), GetWorkBudget.builder().setItems(20L).setBytes(20L).build()); - verify(windmillStreamSender, never()).adjustBudget(anyLong(), anyLong()); + verify(getWorkBudgetSpender, never()).adjustBudget(anyLong(), anyLong()); } @Test @@ -130,12 +103,12 @@ public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHig GetWorkBudget streamRemainingBudget = GetWorkBudget.builder().setItems(1L).setBytes(10L).build(); GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(0L) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq(totalGetWorkBudget.items() - streamRemainingBudget.items()), eq(totalGetWorkBudget.bytes() - streamRemainingBudget.bytes())); @@ -148,12 +121,12 @@ public void testDistributeBudget_doesNotAdjustStreamBudgetWhenRemainingBudgetHig GetWorkBudget.builder().setItems(1L).setBytes(10L).build(); GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); long activeWorkItemsAndBytes = 2L; - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(activeWorkItemsAndBytes) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq( totalGetWorkBudget.items() @@ -167,12 +140,12 @@ public void testDistributeBudget_adjustsStreamBudgetWhenRemainingByteBudgetTooLo GetWorkBudget streamRemainingBudget = GetWorkBudget.builder().setItems(10L).setBytes(1L).build(); GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(0L) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq(totalGetWorkBudget.items() - streamRemainingBudget.items()), eq(totalGetWorkBudget.bytes() - streamRemainingBudget.bytes())); @@ -186,12 +159,12 @@ public void testDistributeBudget_adjustsStreamBudgetWhenRemainingByteBudgetTooLo GetWorkBudget totalGetWorkBudget = GetWorkBudget.builder().setItems(10L).setBytes(10L).build(); long activeWorkItemsAndBytes = 2L; - WindmillStreamSender windmillStreamSender = - spy(createWindmillStreamSender(streamRemainingBudget)); + GetWorkBudgetSpender getWorkBudgetSpender = + spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(streamRemainingBudget)); createBudgetDistributor(activeWorkItemsAndBytes) - .distributeBudget(ImmutableList.of(windmillStreamSender), totalGetWorkBudget); + .distributeBudget(ImmutableList.of(getWorkBudgetSpender), totalGetWorkBudget); - verify(windmillStreamSender, times(1)) + verify(getWorkBudgetSpender, times(1)) .adjustBudget( eq(totalGetWorkBudget.items() - streamRemainingBudget.items()), eq( @@ -203,9 +176,9 @@ public void testDistributeBudget_adjustsStreamBudgetWhenRemainingByteBudgetTooLo @Test public void testDistributeBudget_distributesBudgetEvenlyIfPossible() { long totalItemsAndBytes = 10L; - List streams = new ArrayList<>(); + List streams = new ArrayList<>(); for (int i = 0; i < totalItemsAndBytes; i++) { - streams.add(spy(createWindmillStreamSender(GetWorkBudget.noBudget()))); + streams.add(spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(GetWorkBudget.noBudget()))); } createBudgetDistributor(0L) .distributeBudget( @@ -225,9 +198,9 @@ public void testDistributeBudget_distributesBudgetEvenlyIfPossible() { @Test public void testDistributeBudget_distributesFairlyWhenNotEven() { long totalItemsAndBytes = 10L; - List streams = new ArrayList<>(); + List streams = new ArrayList<>(); for (int i = 0; i < 3; i++) { - streams.add(spy(createWindmillStreamSender(GetWorkBudget.noBudget()))); + streams.add(spy(createGetWorkBudgetOwnerWithRemainingBudgetOf(GetWorkBudget.noBudget()))); } createBudgetDistributor(0L) .distributeBudget( @@ -244,24 +217,17 @@ public void testDistributeBudget_distributesFairlyWhenNotEven() { .adjustBudget(eq(itemsAndBytesPerStream), eq(itemsAndBytesPerStream))); } - private WindmillStreamSender createWindmillStreamSender(GetWorkBudget getWorkBudget) { - return WindmillStreamSender.create( - WindmillConnection.builder().setStub(stub).build(), - Windmill.GetWorkRequest.newBuilder() - .setClientId(1L) - .setJobId("job") - .setProjectId("project") - .build(), - getWorkBudget, - GrpcWindmillStreamFactory.of( - JobHeader.newBuilder() - .setJobId("job") - .setProjectId("project") - .setWorkerId("worker") - .build()) - .build(), - (workItem, watermarks, processingContext, ackWorkItemQueued, getWorkStreamLatencies) -> {}, - ignored -> mock(GetDataClient.class), - ignored -> mock(WorkCommitter.class)); + private GetWorkBudgetSpender createGetWorkBudgetOwnerWithRemainingBudgetOf( + GetWorkBudget getWorkBudget) { + return spy( + new GetWorkBudgetSpender() { + @Override + public void adjustBudget(long itemsDelta, long bytesDelta) {} + + @Override + public GetWorkBudget remainingBudget() { + return getWorkBudget; + } + }); } } From bfc64d5c14adea209364d48b0fe9b4e8ba6eaab5 Mon Sep 17 00:00:00 2001 From: scwhittle Date: Mon, 5 Aug 2024 11:34:45 +0200 Subject: [PATCH 02/21] Fix error when ActiveWorkRefresher processed empty heartbeat map. (#32078) --- .../work/refresh/ActiveWorkRefresher.java | 3 ++ .../work/refresh/ActiveWorkRefresherTest.java | 38 +++++++++++++++++-- 2 files changed, 37 insertions(+), 4 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java index 499d2e5b6943..781285def020 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresher.java @@ -130,6 +130,9 @@ private void refreshActiveWork() { Instant refreshDeadline = clock.get().minus(Duration.millis(activeWorkRefreshPeriodMillis)); Map heartbeatsBySender = aggregateHeartbeatsBySender(refreshDeadline); + if (heartbeatsBySender.isEmpty()) { + return; + } List> fanOutRefreshActiveWork = new ArrayList<>(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java index 9dce3392c60c..5efb2421fe60 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/work/refresh/ActiveWorkRefresherTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.dataflow.worker.windmill.work.refresh; import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.*; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; @@ -194,10 +195,13 @@ public void testActiveWorkRefresh() throws InterruptedException { assertThat(heartbeatRequests) .comparingElementsUsing( Correspondence.from( - (Windmill.HeartbeatRequest h, Work w) -> - h.getWorkToken() == w.getWorkItem().getWorkToken() - && h.getCacheToken() == w.getWorkItem().getWorkToken() - && h.getShardingKey() == w.getWorkItem().getShardingKey(), + (Windmill.HeartbeatRequest h, Work w) -> { + assert h != null; + assert w != null; + return h.getWorkToken() == w.getWorkItem().getWorkToken() + && h.getCacheToken() == w.getWorkItem().getWorkToken() + && h.getShardingKey() == w.getWorkItem().getShardingKey(); + }, "heartbeatRequest's and Work's workTokens, cacheTokens, and shardingKeys should be equal.")) .containsExactlyElementsIn(work); } @@ -207,6 +211,32 @@ public void testActiveWorkRefresh() throws InterruptedException { workIsProcessed.countDown(); } + @Test + public void testEmptyActiveWorkRefresh() throws InterruptedException { + int activeWorkRefreshPeriodMillis = 100; + + List computations = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + ComputationState computationState = createComputationState(i); + computations.add(computationState); + } + + CountDownLatch heartbeatsSent = new CountDownLatch(1); + TestClock fakeClock = new TestClock(Instant.now()); + ActiveWorkRefresher activeWorkRefresher = + createActiveWorkRefresher( + fakeClock::now, + activeWorkRefreshPeriodMillis, + 0, + () -> computations, + heartbeats -> heartbeatsSent::countDown); + + activeWorkRefresher.start(); + fakeClock.advance(Duration.millis(activeWorkRefreshPeriodMillis * 2)); + assertFalse(heartbeatsSent.await(500, TimeUnit.MILLISECONDS)); + activeWorkRefresher.stop(); + } + @Test public void testInvalidateStuckCommits() throws InterruptedException { int stuckCommitDurationMillis = 100; From 80ae93217c5ac74e41cbedaeea7806fb0f05c2a9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 5 Aug 2024 09:49:55 -0700 Subject: [PATCH 03/21] Minor optimization for the common case of merging empty string sets. (#31803) --- .../runners/core/metrics/StringSetData.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java index 93dfb8e3ebc8..466d4ad46eb6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/StringSetData.java @@ -19,7 +19,6 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; -import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -50,12 +49,16 @@ public static StringSetData empty() { * Combines this {@link StringSetData} with other, both original StringSetData are left intact. */ public StringSetData combine(StringSetData other) { - // do not merge other on this as this StringSetData might hold an immutable set like in case - // of EmptyStringSetData - Set combined = new HashSet<>(); - combined.addAll(this.stringSet()); - combined.addAll(other.stringSet()); - return StringSetData.create(combined); + if (this.stringSet().isEmpty()) { + return other; + } else if (other.stringSet().isEmpty()) { + return this; + } else { + ImmutableSet.Builder combined = ImmutableSet.builder(); + combined.addAll(this.stringSet()); + combined.addAll(other.stringSet()); + return StringSetData.create(combined.build()); + } } /** From 5b2bfe96f83a5631c3a8d5c3b92a0f695ffe2d7d Mon Sep 17 00:00:00 2001 From: Damon Date: Mon, 5 Aug 2024 10:25:37 -0700 Subject: [PATCH 04/21] [Prism] Enable an artifact resolver for the Prism runner (#32058) * Enable an ArtifactResolver for the Prism runner * Rename class * spotlessApply * Fix Builder instantiation --- .../runners/prism/PrismArtifactResolver.java | 110 ++++++++++++++++++ .../prism/PrismArtifactResolverTest.java | 45 +++++++ 2 files changed, 155 insertions(+) create mode 100644 runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java create mode 100644 runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java diff --git a/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java new file mode 100644 index 000000000000..db56bc6047ca --- /dev/null +++ b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactResolver.java @@ -0,0 +1,110 @@ +/* + * 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.prism; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.construction.DefaultArtifactResolver; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.util.construction.SdkComponents; + +/** + * The {@link PrismArtifactResolver} converts a {@link Pipeline} to a {@link RunnerApi.Pipeline} via + * resolving {@link RunnerApi.ArtifactInformation}. + */ +@AutoValue +abstract class PrismArtifactResolver { + + /** + * Instantiates a {@link PrismArtifactResolver} from the {@param pipeline}, applying defaults to + * the remaining dependencies. + */ + static PrismArtifactResolver of(Pipeline pipeline) { + return PrismArtifactResolver.builder().setPipeline(pipeline).build(); + } + + static Builder builder() { + return new AutoValue_PrismArtifactResolver.Builder(); + } + + /** + * Converts the {@link #getPipeline()} using {@link PipelineTranslation#toProto} and {@link + * #getDelegate()}'s {@link + * org.apache.beam.sdk.util.construction.ArtifactResolver#resolveArtifacts}. + */ + RunnerApi.Pipeline resolvePipelineProto() { + RunnerApi.Pipeline result = PipelineTranslation.toProto(getPipeline(), getSdkComponents()); + return getDelegate().resolveArtifacts(result); + } + + /** + * {@link PrismArtifactResolver} delegates to {@link + * org.apache.beam.sdk.util.construction.ArtifactResolver} to transform {@link + * RunnerApi.ArtifactInformation}. Defaults to {@link DefaultArtifactResolver#INSTANCE} if not + * set. + */ + abstract org.apache.beam.sdk.util.construction.ArtifactResolver getDelegate(); + + /** The {@link Pipeline} from which {@link PrismArtifactResolver#resolvePipelineProto()}. */ + abstract Pipeline getPipeline(); + + /** + * SDK objects that will be represented by {@link + * org.apache.beam.model.pipeline.v1.RunnerApi.Components}. Instantiated via {@link + * SdkComponents#create(PipelineOptions)} by default, where {@link PipelineOptions} are acquired + * from {@link #getPipeline}'s {@link Pipeline#getOptions}. + */ + abstract SdkComponents getSdkComponents(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setDelegate( + org.apache.beam.sdk.util.construction.ArtifactResolver artifactResolver); + + abstract Optional getDelegate(); + + abstract Builder setSdkComponents(SdkComponents sdkComponents); + + abstract Optional getSdkComponents(); + + abstract Builder setPipeline(Pipeline pipeline); + + abstract Optional getPipeline(); + + abstract PrismArtifactResolver autoBuild(); + + final PrismArtifactResolver build() { + if (!getDelegate().isPresent()) { + setDelegate(DefaultArtifactResolver.INSTANCE); + } + + if (!getSdkComponents().isPresent()) { + checkState(getPipeline().isPresent()); + setSdkComponents(SdkComponents.create(getPipeline().get().getOptions())); + } + + return autoBuild(); + } + } +} diff --git a/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.java new file mode 100644 index 000000000000..ef4646f02347 --- /dev/null +++ b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactResolverTest.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.prism; + +import static com.google.common.truth.Truth.assertThat; + +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.Impulse; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link PrismArtifactResolver}. */ +@RunWith(JUnit4.class) +public class PrismArtifactResolverTest { + @Test + public void resolvesPipeline() { + Pipeline pipeline = Pipeline.create(); + pipeline.apply(Impulse.create()); + PrismArtifactResolver underTest = PrismArtifactResolver.of(pipeline); + RunnerApi.Pipeline pipelineProto = underTest.resolvePipelineProto(); + RunnerApi.Components components = pipelineProto.getComponents(); + assertThat(components.getTransformsMap()).containsKey("Impulse"); + assertThat(components.getCodersMap()).containsKey("ByteArrayCoder"); + assertThat(components.getEnvironmentsMap()) + .containsKey(BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER)); + } +} From fb49e9644a4b81bdca339d98181c6f21256d474a Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 5 Aug 2024 18:00:08 -0400 Subject: [PATCH 05/21] Fix load test dataproc cluster name exceeded allowed length (#32062) --- .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml | 2 +- .github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml | 2 +- .../workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml index 5ded71a7652a..e2afb2e2cfd7 100644 --- a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -48,7 +48,7 @@ env: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a - CLUSTER_NAME: beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} + CLUSTER_NAME: beam-loadtests-py-cogbk-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml index 061a1b9e210e..bae2f9f82ee1 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Batch.yml @@ -48,7 +48,7 @@ env: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a - CLUSTER_NAME: beam-loadtests-python-pardo-flink-batch-${{ github.run_id }} + CLUSTER_NAME: beam-loadtests-py-pardo-flink-batch-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar diff --git a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml index bec926ab9656..4485b7187f80 100644 --- a/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_ParDo_Flink_Streaming.yml @@ -48,7 +48,7 @@ env: INFLUXDB_USER: ${{ secrets.INFLUXDB_USER }} INFLUXDB_USER_PASSWORD: ${{ secrets.INFLUXDB_USER_PASSWORD }} GCLOUD_ZONE: us-central1-a - CLUSTER_NAME: beam-loadtests-python-pardo-flink-stream-${{ github.run_id }} + CLUSTER_NAME: beam-loadtests-py-pardo-flink-stream-${{ github.run_id }} GCS_BUCKET: gs://beam-flink-cluster FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar From c60623524ae9998cdfb8bfb1985f218e7dfa823a Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 6 Aug 2024 10:51:42 -0400 Subject: [PATCH 06/21] Beam Website Updates for 2.58.0 Release (#31925) * Beam Website Updates for 2.58.0 Release * Update 2.57.0 links to archive links * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * move solace io to highlights * Add SpannerIO breaking change * fix urls * Update CHANGES.md Co-authored-by: tvalentyn * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Update release date * add release date to changes.md --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> Co-authored-by: tvalentyn --- CHANGES.md | 21 +-- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.58.0.md | 130 ++++++++++++++++++ .../site/content/en/get-started/downloads.md | 13 +- 4 files changed, 143 insertions(+), 23 deletions(-) create mode 100644 website/www/site/content/en/blog/beam-2.58.0.md diff --git a/CHANGES.md b/CHANGES.md index b127599ae0aa..7f12b5334260 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -88,15 +88,10 @@ * ([#X](https://github.com/apache/beam/issues/X)). -# [2.58.0] - Unreleased +# [2.58.0] - 2024-08-06 ## Highlights -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). - -## I/Os - * Support for [Solace](https://solace.com/) source (`SolaceIO.Read`) added (Java) ([#31440](https://github.com/apache/beam/issues/31440)). ## New Features / Improvements @@ -110,25 +105,13 @@ ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). * [IcebergIO] IcebergCatalogConfig was changed to support specifying catalog properties in a key-store fashion ([#31726](https://github.com/apache/beam/pull/31726)) * [SpannerIO] Added validation that query and table cannot be specified at the same time for SpannerIO.read(). Previously withQuery overrides withTable, if set ([#24956](https://github.com/apache/beam/issues/24956)). -## Deprecations - -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). - ## Bugfixes * [BigQueryIO] Fixed a bug in batch Storage Write API that frequently exhausted concurrent connections quota ([#31710](https://github.com/apache/beam/pull/31710)) -* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). - -## Security Fixes -* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). - -## Known Issues - -* ([#X](https://github.com/apache/beam/issues/X)). +* Fixed a logging issue where Python worker dependency installation logs sometimes were not emitted in a timely manner ([#31977](https://github.com/apache/beam/pull/31977)) # [2.57.0] - 2024-06-26 diff --git a/website/www/site/config.toml b/website/www/site/config.toml index 7fe6df7a2c7a..6675cf418bdd 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.57.0" +release_latest = "2.58.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/blog/beam-2.58.0.md b/website/www/site/content/en/blog/beam-2.58.0.md new file mode 100644 index 000000000000..603403cd7fdb --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.58.0.md @@ -0,0 +1,130 @@ +--- +title: "Apache Beam 2.58.0" +date: 2024-08-06 13:00:00 -0800 +categories: + - blog + - release +authors: + - jrmccluskey +--- + + +We are happy to present the new 2.58.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#2580-2024-08-06) for this release. + + + +For more information about changes in 2.58.0, check out the [detailed release notes](https://github.com/apache/beam/milestone/22). + +## I/Os + +* Support for [Solace](https://solace.com/) source (`SolaceIO.Read`) added (Java) ([#31440](https://github.com/apache/beam/issues/31440)). + +## New Features / Improvements + +* Multiple RunInference instances can now share the same model instance by setting the model_identifier parameter (Python) ([#31665](https://github.com/apache/beam/issues/31665)). +* Added options to control the number of Storage API multiplexing connections ([#31721](https://github.com/apache/beam/pull/31721)) +* [BigQueryIO] Better handling for batch Storage Write API when it hits AppendRows throughput quota ([#31837](https://github.com/apache/beam/pull/31837)) +* [IcebergIO] All specified catalog properties are passed through to the connector ([#31726](https://github.com/apache/beam/pull/31726)) +* Removed a third-party LGPL dependency from the Go SDK ([#31765](https://github.com/apache/beam/issues/31765)). +* Support for `MapState` and `SetState` when using Dataflow Runner v1 with Streaming Engine (Java) ([[#18200](https://github.com/apache/beam/issues/18200)]) + +## Breaking Changes + +* [IcebergIO] `IcebergCatalogConfig` was changed to support specifying catalog properties in a key-store fashion ([#31726](https://github.com/apache/beam/pull/31726)) +* [SpannerIO] Added validation that query and table cannot be specified at the same time for `SpannerIO.read()`. Previously `withQuery` overrides `withTable`, if set ([#24956](https://github.com/apache/beam/issues/24956)). + +## Bug fixes + +* [BigQueryIO] Fixed a bug in batch Storage Write API that frequently exhausted concurrent connections quota ([#31710](https://github.com/apache/beam/pull/31710)) + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.58.0 release. Thank you to all contributors! + +Ahmed Abualsaud + +Ahmet Altay + +Alexandre Moueddene + +Alexey Romanenko + +Andrew Crites + +Bartosz Zablocki + +Celeste Zeng + +Chamikara Jayalath + +Clay Johnson + +Damon Douglass + +Danny McCormick + +Dilnaz Amanzholova + +Florian Bernard + +Francis O'Hara + +George Ma + +Israel Herraiz + +Jack McCluskey + +Jaehyeon Kim + +James Roseman + +Kenneth Knowles + +Maciej Szwaja + +Michel Davit + +Minh Son Nguyen + +Naireen + +Niel Markwick + +Oliver Cardoza + +Robert Bradshaw + +Robert Burke + +Rohit Sinha + +S. Veyrié + +Sam Whittle + +Shunping Huang + +Svetak Sundhar + +TongruiLi + +Tony Tang + +Valentyn Tymofieiev + +Vitaly Terentyev + +Yi Hu \ No newline at end of file diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 8f3b92ef9f2a..b7db1ddd65b6 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -96,10 +96,17 @@ versions denoted `0.x.y`. ## Releases +### 2.58.0 (2024-08-06) +Official [source code download](https://downloads.apache.org/beam/2.58.0/apache-beam-2.58.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.58.0/apache-beam-2.58.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.58.0/apache-beam-2.58.0-source-release.zip.asc). + +[Release notes](https://github.com/apache/beam/releases/tag/v2.58.0) + ### 2.57.0 (2024-06-26) -Official [source code download](https://downloads.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip). +[SHA-512](https://archive.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.sha512). +[signature](https://archive.apache.org/beam/2.57.0/apache-beam-2.57.0-source-release.zip.asc). [Release notes](https://github.com/apache/beam/releases/tag/v2.57.0) From d09c3237c8aa1ed48351046fe61bc0cc8794521a Mon Sep 17 00:00:00 2001 From: atask-g Date: Tue, 6 Aug 2024 11:53:36 -0400 Subject: [PATCH 07/21] Added support for the TOKENLIST type in Spanner (#32038) --- .../org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java | 3 +++ .../org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java index 3fd09c63da79..fa44cadeba0a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java @@ -184,6 +184,9 @@ private static Type parseSpannerType(String spannerType, Dialect dialect) { if (spannerType.startsWith("BYTES")) { return Type.bytes(); } + if ("TOKENLIST".equals(spannerType)) { + return Type.bytes(); + } if ("TIMESTAMP".equals(spannerType)) { return Type.timestamp(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java index 166df1704ca8..1e89326d1e8c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchemaTest.java @@ -40,16 +40,18 @@ public void testSingleTable() throws Exception { .addColumn("test", "jsonVal", "JSON") .addColumn("test", "protoVal", "PROTO") .addColumn("test", "enumVal", "ENUM") + .addColumn("test", "tokens", "TOKENLIST") .build(); assertEquals(1, schema.getTables().size()); - assertEquals(6, schema.getColumns("test").size()); + assertEquals(7, schema.getColumns("test").size()); assertEquals(1, schema.getKeyParts("test").size()); assertEquals(Type.json(), schema.getColumns("test").get(3).getType()); assertEquals( Type.proto("customer.app.TestMessage"), schema.getColumns("test").get(4).getType()); assertEquals( Type.protoEnum("customer.app.TestEnum"), schema.getColumns("test").get(5).getType()); + assertEquals(Type.bytes(), schema.getColumns("test").get(6).getType()); } @Test From e9b5dc69532865e4ec20faa13a1ff88552bc50ae Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Aug 2024 12:39:03 -0400 Subject: [PATCH 08/21] Enforce java.nio.charset.StandardCharsets against guava Charsets (#32083) * Enforce java.nio.charset.StandardCharsets against guava Charsets * Fix dependency analyzeClassesDependencies --- .../transforms/FormatTransform.java | 5 ++-- .../streaming/io/StreamingImpulseSource.java | 5 ++-- .../flink/FlinkJobServerDriverTest.java | 12 ++++++---- ...FlinkPipelineExecutionEnvironmentTest.java | 4 ++-- .../runners/flink/FlinkSubmissionTest.java | 4 ++-- .../ExecutableStageDoFnOperatorTest.java | 9 ++++--- .../wrappers/streaming/FlinkKeyUtilsTest.java | 4 ++-- .../dataflow/DataflowPipelineTranslator.java | 7 +++--- .../runners/dataflow/worker/graph/Nodes.java | 6 ++--- .../GroupingShuffleEntryIteratorTest.java | 7 +++--- .../state/WindmillStateInternalsTest.java | 11 +++++---- .../state/WindmillStateReaderTest.java | 8 +++---- .../artifact/ArtifactStagingService.java | 4 ++-- .../state/StateRequestHandlers.java | 5 ++-- .../ArtifactRetrievalServiceTest.java | 4 ++-- .../testing/TestUniversalRunner.java | 5 ++-- .../resources/beam/checkstyle/checkstyle.xml | 8 +++++++ .../providers/LoggingTransformProvider.java | 4 ++-- .../transforms/errorhandling/BadRecord.java | 6 ++--- .../ByteBuddyOnTimerInvokerFactory.java | 6 ++--- .../resourcehints/ResourceHints.java | 8 +++---- .../sdk/coders/StructuralByteArrayTest.java | 10 ++++---- .../apache/beam/sdk/io/FileBasedSinkTest.java | 2 +- .../org/apache/beam/sdk/io/FileIOTest.java | 6 ++--- .../apache/beam/sdk/io/TFRecordIOTest.java | 7 +++--- .../apache/beam/sdk/io/TextIOReadTest.java | 8 +++---- .../apache/beam/sdk/io/TextIOWriteTest.java | 6 ++--- .../sdk/io/TextRowCountEstimatorTest.java | 10 ++++---- .../apache/beam/sdk/io/WriteFilesTest.java | 5 ++-- .../options/PipelineOptionsFactoryTest.java | 24 +++++++++---------- .../sdk/schemas/SchemaTranslationTest.java | 5 ++-- ...fferedElementCountingOutputStreamTest.java | 6 ++--- .../util/ExposedByteArrayInputStreamTest.java | 6 ++--- .../ExposedByteArrayOutputStreamTest.java | 4 ++-- .../beam/sdk/util/SerializableUtilsTest.java | 4 ++-- .../service/ExpansionServiceTest.java | 4 ++-- .../avro/AvroGenericCoderTranslator.java | 6 ++--- .../sdk/extensions/avro/io/AvroIOTest.java | 6 ++--- .../python/PythonExternalTransform.java | 5 ++-- .../sdk/extensions/python/PythonService.java | 5 ++-- sdks/java/extensions/sql/jdbc/build.gradle | 2 +- .../sdk/extensions/sql/jdbc/BeamSqlLine.java | 6 ++--- .../provider/text/TextTableProviderTest.java | 20 +++++++++------- .../sdk/io/clickhouse/ClickHouseWriter.java | 4 ++-- .../ContextualTextIOTest.java | 16 ++++++------- ...PubsubReadSchemaTransformProviderTest.java | 6 ++--- .../sdk/io/kafka/ReadFromKafkaDoFnTest.java | 6 ++--- .../apache/beam/sdk/tpcds/QueryReader.java | 4 ++-- .../beam/sdk/tpcds/SqlTransformRunner.java | 4 ++-- .../beam/sdk/tpcds/TableSchemaJSONLoader.java | 4 ++-- .../TransformServiceLauncherTest.java | 10 ++++---- 51 files changed, 182 insertions(+), 161 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java index 2d9089fcd29a..296d7e7d2409 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples.complete.kafkatopubsub.transforms; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClass; @@ -37,7 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.kafka.common.serialization.StringDeserializer; @@ -120,7 +120,8 @@ public PDone expand(PCollection input) { MapElements.into(TypeDescriptor.of(PubsubMessage.class)) .via( (String json) -> - new PubsubMessage(json.getBytes(Charsets.UTF_8), ImmutableMap.of()))) + new PubsubMessage( + json.getBytes(StandardCharsets.UTF_8), ImmutableMap.of()))) .apply( "writePubsubMessagesToPubSub", PubsubIO.writeMessages().to(options.getOutputTopic())); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java index 8f21e42d61e6..871d7a5a3989 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +60,8 @@ public void run(SourceContext> ctx) { while (running && (messageCount == 0 || count < subtaskCount)) { synchronized (ctx.getCheckpointLock()) { ctx.collect( - WindowedValue.valueInGlobalWindow(String.valueOf(count).getBytes(Charsets.UTF_8))); + WindowedValue.valueInGlobalWindow( + String.valueOf(count).getBytes(StandardCharsets.UTF_8))); count++; } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java index 4a628eeb4fdf..22516cbc9633 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobServerDriverTest.java @@ -25,7 +25,7 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; /** Tests for {@link FlinkJobServerDriver}. */ @@ -104,7 +104,7 @@ public void testJobServerDriver() throws Exception { boolean success = false; while (!success) { newErr.flush(); - String output = baos.toString(Charsets.UTF_8.name()); + String output = baos.toString(StandardCharsets.UTF_8.name()); if (output.contains("JobService started on localhost:") && output.contains("ArtifactStagingService started on localhost:") && output.contains("ExpansionService started on localhost:")) { @@ -114,7 +114,8 @@ public void testJobServerDriver() throws Exception { } } assertThat(driver.getJobServerUrl(), is(not(nullValue()))); - assertThat(baos.toString(Charsets.UTF_8.name()), containsString(driver.getJobServerUrl())); + assertThat( + baos.toString(StandardCharsets.UTF_8.name()), containsString(driver.getJobServerUrl())); assertThat(driverThread.isAlive(), is(true)); } catch (Throwable t) { // restore to print exception @@ -149,7 +150,7 @@ public void testJobServerDriverWithoutExpansionService() throws Exception { boolean success = false; while (!success) { newErr.flush(); - String output = baos.toString(Charsets.UTF_8.name()); + String output = baos.toString(StandardCharsets.UTF_8.name()); if (output.contains("JobService started on localhost:") && output.contains("ArtifactStagingService started on localhost:")) { success = true; @@ -161,7 +162,8 @@ public void testJobServerDriverWithoutExpansionService() throws Exception { } } assertThat(driver.getJobServerUrl(), is(not(nullValue()))); - assertThat(baos.toString(Charsets.UTF_8.name()), containsString(driver.getJobServerUrl())); + assertThat( + baos.toString(StandardCharsets.UTF_8.name()), containsString(driver.getJobServerUrl())); assertThat(driverThread.isAlive(), is(true)); } catch (Throwable t) { // restore to print exception diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java index 9d898ed53a89..3b92c282c38a 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java @@ -38,6 +38,7 @@ import java.lang.reflect.Method; import java.net.MalformedURLException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -56,7 +57,6 @@ import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.resources.PipelineResources; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.RemoteEnvironment; @@ -373,7 +373,7 @@ public void processElement(ProcessContext ctx) { } replacementStdErr.flush(); assertThat( - new String(byteArrayOutputStream.toByteArray(), Charsets.UTF_8), + new String(byteArrayOutputStream.toByteArray(), StandardCharsets.UTF_8), containsString( "UnboundedSources present which rely on checkpointing, but checkpointing is disabled.")); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java index 22a9ce4f39ab..cf860717def3 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java @@ -20,6 +20,7 @@ import java.io.File; import java.lang.reflect.Field; import java.lang.reflect.Modifier; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.security.Permission; import java.util.Collection; @@ -30,7 +31,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.construction.resources.PipelineResources; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.collect.Iterables; @@ -200,7 +200,7 @@ private static void prepareEnvironment() throws Exception { RestOptions.PORT.key(), flinkCluster.getRestPort()); - Files.write(file.toPath(), config.getBytes(Charsets.UTF_8)); + Files.write(file.toPath(), config.getBytes(StandardCharsets.UTF_8)); // Create a new environment with the location of the Flink config for CliFrontend ImmutableMap newEnv = diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index cf5b2b555124..2eb0545b7794 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -103,7 +103,6 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.Struct; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.collect.Iterables; @@ -818,7 +817,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing BagState state = // State from the SDK Harness is stored as ByteStrings operator.keyedStateInternals.state( stateNamespace, StateTags.bag(stateId, ByteStringCoder.of())); - state.add(ByteString.copyFrom("userstate".getBytes(Charsets.UTF_8))); + state.add(ByteString.copyFrom("userstate".getBytes(StandardCharsets.UTF_8))); assertThat(testHarness.numKeyedStateEntries(), is(1)); // user timer that fires after the end of the window and after state cleanup @@ -966,7 +965,7 @@ public void testEnsureStateCleanupOnFinalWatermark() throws Exception { BagState state = // State from the SDK Harness is stored as ByteStrings operator.keyedStateInternals.state( stateNamespace, StateTags.bag(stateId, ByteStringCoder.of())); - state.add(ByteString.copyFrom("userstate".getBytes(Charsets.UTF_8))); + state.add(ByteString.copyFrom("userstate".getBytes(StandardCharsets.UTF_8))); // No timers have been set for cleanup assertThat(testHarness.numEventTimeTimers(), is(0)); // State has been created @@ -988,8 +987,8 @@ public void testCacheTokenHandling() throws Exception { new ExecutableStageDoFnOperator.BagUserStateFactory<>( test, stateBackend, NoopLock.get(), null); - ByteString key1 = ByteString.copyFrom("key1", Charsets.UTF_8); - ByteString key2 = ByteString.copyFrom("key2", Charsets.UTF_8); + ByteString key1 = ByteString.copyFrom("key1", StandardCharsets.UTF_8); + ByteString key2 = ByteString.copyFrom("key2", StandardCharsets.UTF_8); Map> userStateMapMock = Mockito.mock(Map.class); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java index cab45632ac55..cdf461b5fde8 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java @@ -22,12 +22,12 @@ import static org.hamcrest.core.Is.is; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Test; /** Tests for {@link FlinkKeyUtils}. */ @@ -66,7 +66,7 @@ public void testCoderContext() throws Exception { @Test @SuppressWarnings("ByteBufferBackingArray") public void testFromEncodedKey() { - ByteString input = ByteString.copyFrom("hello world".getBytes(Charsets.UTF_8)); + ByteString input = ByteString.copyFrom("hello world".getBytes(StandardCharsets.UTF_8)); ByteBuffer encodedKey = FlinkKeyUtils.fromEncodedKey(input); assertThat(encodedKey.array(), is(input.toByteArray())); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index f905e136e83b..1fedcd8f3a29 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -42,6 +42,7 @@ import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.Step; import com.google.api.services.dataflow.model.WorkerPool; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; @@ -110,7 +111,6 @@ import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.commons.codec.EncoderException; import org.apache.commons.codec.net.PercentCodec; @@ -618,7 +618,7 @@ static class StepTranslator implements StepTranslationContext { // For compatibility with URL encoding implementations that represent space as +, // always encode + as %2b even though we don't encode space as +. private final PercentCodec percentCodec = - new PercentCodec("+".getBytes(Charsets.US_ASCII), false); + new PercentCodec("+".getBytes(StandardCharsets.US_ASCII), false); private StepTranslator(Translator translator, Step step) { this.translator = translator; @@ -764,7 +764,8 @@ private void addResourceHints(ResourceHints hints) { try { urlEncodedHints.put( entry.getKey(), - new String(percentCodec.encode(entry.getValue().toBytes()), Charsets.US_ASCII)); + new String( + percentCodec.encode(entry.getValue().toBytes()), StandardCharsets.US_ASCII)); } catch (EncoderException e) { // Should never happen. throw new RuntimeException("Invalid value for resource hint: " + entry.getKey(), e); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java index 6092d0d64de5..d82432417000 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java @@ -29,11 +29,11 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.nio.charset.StandardCharsets; import org.apache.beam.runners.dataflow.worker.util.common.worker.Operation; import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputReceiver; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.gcp.util.Transport; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; /** Container class for different types of network nodes. All nodes only have reference equality. */ @@ -59,7 +59,7 @@ private static String toStringWithTrimmedLiterals(GenericJson json) { ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); final JsonGenerator baseGenerator = MoreObjects.firstNonNull(json.getFactory(), Transport.getJsonFactory()) - .createJsonGenerator(byteStream, Charsets.UTF_8); + .createJsonGenerator(byteStream, StandardCharsets.UTF_8); JsonGenerator generator = new JsonGenerator() { @Override @@ -164,7 +164,7 @@ public void enablePrettyPrint() throws IOException { generator.enablePrettyPrint(); generator.serialize(json); generator.flush(); - return byteStream.toString(Charsets.UTF_8.name()); + return byteStream.toString(StandardCharsets.UTF_8.name()); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java index 2421d7faf824..8c6a003cb72b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.After; @@ -130,10 +129,10 @@ private void setCurrentExecutionState(String mockOriginalName) { private static ShuffleEntry shuffleEntry(String key, String value) { return new ShuffleEntry( /* use key itself as position */ - ByteArrayShufflePosition.of(key.getBytes(Charsets.UTF_8)), - ByteString.copyFrom(key.getBytes(Charsets.UTF_8)), + ByteArrayShufflePosition.of(key.getBytes(StandardCharsets.UTF_8)), + ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)), ByteString.copyFrom(new byte[0]), - ByteString.copyFrom(value.getBytes(Charsets.UTF_8))); + ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8))); } @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 33e47623cd0e..d06ed0f526c7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -80,7 +80,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -225,7 +224,7 @@ public void resetUnderTest() { .forComputation("comp") .forKey( WindmillComputationKey.create( - "comp", ByteString.copyFrom("dummyKey", Charsets.UTF_8), 123), + "comp", ByteString.copyFrom("dummyKey", StandardCharsets.UTF_8), 123), 17L, workToken) .forFamily(STATE_FAMILY), @@ -240,7 +239,7 @@ public void resetUnderTest() { .forComputation("comp") .forKey( WindmillComputationKey.create( - "comp", ByteString.copyFrom("dummyNewKey", Charsets.UTF_8), 123), + "comp", ByteString.copyFrom("dummyNewKey", StandardCharsets.UTF_8), 123), 17L, workToken) .forFamily(STATE_FAMILY), @@ -255,7 +254,7 @@ public void resetUnderTest() { .forComputation("comp") .forKey( WindmillComputationKey.create( - "comp", ByteString.copyFrom("dummyNewKey", Charsets.UTF_8), 123), + "comp", ByteString.copyFrom("dummyNewKey", StandardCharsets.UTF_8), 123), 17L, workToken) .forFamily(STATE_FAMILY), @@ -2004,7 +2003,9 @@ false, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) } // clear cache and recreate multimapState - cache.forComputation("comp").invalidate(ByteString.copyFrom("dummyKey", Charsets.UTF_8), 123); + cache + .forComputation("comp") + .invalidate(ByteString.copyFrom("dummyKey", StandardCharsets.UTF_8), 123); resetUnderTest(); multimapState = underTest.state(NAMESPACE, addr); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 8dbfc35192b7..b06d88bf4bc4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -27,6 +27,7 @@ import com.google.api.client.util.Lists; import com.google.common.collect.Maps; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; @@ -48,7 +49,6 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; @@ -1151,8 +1151,8 @@ public void testReadSortedListWithContinuations() throws Exception { .addFetchRanges(SortedListRange.newBuilder().setStart(beginning).setLimit(end)) .setFetchMaxBytes(WindmillStateReader.MAX_ORDERED_LIST_BYTES)); - final ByteString CONT_1 = ByteString.copyFrom("CONTINUATION_1", Charsets.UTF_8); - final ByteString CONT_2 = ByteString.copyFrom("CONTINUATION_2", Charsets.UTF_8); + final ByteString CONT_1 = ByteString.copyFrom("CONTINUATION_1", StandardCharsets.UTF_8); + final ByteString CONT_2 = ByteString.copyFrom("CONTINUATION_2", StandardCharsets.UTF_8); Windmill.KeyedGetDataResponse.Builder response1 = Windmill.KeyedGetDataResponse.newBuilder() .setKey(DATA_KEY) @@ -1327,7 +1327,7 @@ public void testReadTagValuePrefixWithContinuations() throws Exception { .setStateFamily(STATE_FAMILY) .setFetchMaxBytes(WindmillStateReader.MAX_TAG_VALUE_PREFIX_BYTES)); - final ByteString CONT = ByteString.copyFrom("CONTINUATION", Charsets.UTF_8); + final ByteString CONT = ByteString.copyFrom("CONTINUATION", StandardCharsets.UTF_8); Windmill.KeyedGetDataResponse.Builder response1 = Windmill.KeyedGetDataResponse.newBuilder() .setKey(DATA_KEY) diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java index a8e5e2ab6a88..8c7a356b9939 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; @@ -56,7 +57,6 @@ import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Status; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.StatusException; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -192,7 +192,7 @@ public void removeStagedArtifacts(String stagingToken) throws IOException { private ResourceId stagingDir(String stagingToken) { return FileSystems.matchNewResource(root, true) .resolve( - Hashing.sha256().hashString(stagingToken, Charsets.UTF_8).toString(), + Hashing.sha256().hashString(stagingToken, StandardCharsets.UTF_8).toString(), ResolveOptions.StandardResolveOptions.RESOLVE_DIRECTORY); } }; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java index e2c45850dba9..4e1c31744c1a 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** @@ -655,7 +655,8 @@ private BagUserStateHandler createHandl } private static BeamFnApi.ProcessBundleRequest.CacheToken createCacheToken() { - ByteString token = ByteString.copyFrom(UUID.randomUUID().toString().getBytes(Charsets.UTF_8)); + ByteString token = + ByteString.copyFrom(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)); return BeamFnApi.ProcessBundleRequest.CacheToken.newBuilder() .setUserState(BeamFnApi.ProcessBundleRequest.CacheToken.UserState.getDefaultInstance()) .setToken(token) diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java index d6b48a936135..4d19e87c3d11 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -33,7 +34,6 @@ import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Before; @@ -74,7 +74,7 @@ private void stageFiles(Map files) throws IOException { for (Map.Entry entry : files.entrySet()) { Files.write( Paths.get(stagingDir.toString(), entry.getKey()), - entry.getValue().getBytes(Charsets.UTF_8)); + entry.getValue().getBytes(StandardCharsets.UTF_8)); } } diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java index 533106869c62..a36c1e8b2efb 100644 --- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java +++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestUniversalRunner.java @@ -21,6 +21,7 @@ import com.google.auto.service.AutoService; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import org.apache.beam.runners.portability.PortableRunner; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; @@ -65,7 +65,8 @@ public PipelineResult run(Pipeline pipeline) { testOptions.setJobEndpoint( "localhost:" + new String( - Files.readAllBytes(Paths.get(localServicePortFilePath)), Charsets.UTF_8) + Files.readAllBytes(Paths.get(localServicePortFilePath)), + StandardCharsets.UTF_8) .trim()); } catch (IOException e) { throw new RuntimeException( diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml index 3c4cfdfbc6f5..5cee5d2f33e2 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/checkstyle.xml @@ -119,6 +119,14 @@ page at http://checkstyle.sourceforge.net/config.html --> + + + + + + + + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java index 25efaeae2a0e..2908171f5c02 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/providers/LoggingTransformProvider.java @@ -19,6 +19,7 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.List; import java.util.Map; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.slf4j.Logger; @@ -166,7 +166,7 @@ private static DoFn createDoFn(Level logLevel, String prefix, Schema r return new DoFn() { @ProcessElement public void processElement(@Element Row row, OutputReceiver out) { - String msg = prefix + new String(fn.apply(row), Charsets.UTF_8); + String msg = prefix + new String(fn.apply(row), StandardCharsets.UTF_8); // Looks like this is the best we can do. // https://stackoverflow.com/questions/2621701/setting-log-level-of-message-at-runtime-in-slf4j switch (logLevel) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java index fd49078350c4..558f912a6b1f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/errorhandling/BadRecord.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.io.PrintStream; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -202,11 +202,11 @@ public abstract static class Builder { public Builder addExceptionStackTrace(Exception exception) throws IOException { ByteArrayOutputStream stream = new ByteArrayOutputStream(); - PrintStream printStream = new PrintStream(stream, false, Charsets.UTF_8.name()); + PrintStream printStream = new PrintStream(stream, false, StandardCharsets.UTF_8.name()); exception.printStackTrace(printStream); printStream.close(); - this.setExceptionStacktrace(new String(stream.toByteArray(), Charsets.UTF_8)); + this.setExceptionStacktrace(new String(stream.toByteArray(), StandardCharsets.UTF_8)); return this; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java index 7b9ac7e15c2e..e318e82513ca 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java @@ -22,6 +22,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ExecutionException; import net.bytebuddy.ByteBuddy; import net.bytebuddy.description.modifier.FieldManifestation; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.transforms.DoFn.TimerId; import org.apache.beam.sdk.transforms.reflect.ByteBuddyDoFnInvokerFactory.DoFnMethodWithExtraParametersDelegation; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.CharMatcher; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.LoadingCache; @@ -191,7 +191,7 @@ public Constructor load(final OnTimerMethodSpecifier onTimerMethodSpecifier) "%s$%s$%s", OnTimerInvoker.class.getSimpleName(), CharMatcher.javaLetterOrDigit().retainFrom(timerId), - BaseEncoding.base64().omitPadding().encode(timerId.getBytes(Charsets.UTF_8))); + BaseEncoding.base64().omitPadding().encode(timerId.getBytes(StandardCharsets.UTF_8))); DynamicType.Builder builder = new ByteBuddy() @@ -241,7 +241,7 @@ public Constructor load(final OnTimerMethodSpecifier onTimerMethodSpecifier) "%s$%s$%s", OnTimerInvoker.class.getSimpleName(), CharMatcher.javaLetterOrDigit().retainFrom(timerId), - BaseEncoding.base64().omitPadding().encode(timerId.getBytes(Charsets.UTF_8))); + BaseEncoding.base64().omitPadding().encode(timerId.getBytes(StandardCharsets.UTF_8))); DynamicType.Builder builder = new ByteBuddy() diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java index 2f034626acd7..527a699568f4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java @@ -19,6 +19,7 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -28,7 +29,6 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.StandardResourceHints; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ProtocolMessageEnum; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -179,7 +179,7 @@ public ResourceHint mergeWithOuter(ResourceHint outer) { @Override public byte[] toBytes() { - return String.valueOf(value).getBytes(Charsets.US_ASCII); + return String.valueOf(value).getBytes(StandardCharsets.US_ASCII); } } @@ -196,7 +196,7 @@ public static String parse(String s) { @Override public byte[] toBytes() { - return value.getBytes(Charsets.US_ASCII); + return value.getBytes(StandardCharsets.US_ASCII); } @Override @@ -254,7 +254,7 @@ public ResourceHint mergeWithOuter(ResourceHint outer) { @Override public byte[] toBytes() { - return String.valueOf(value).getBytes(Charsets.US_ASCII); + return String.valueOf(value).getBytes(StandardCharsets.US_ASCII); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java index bd8fdd84fb09..cb0845796fe9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuralByteArrayTest.java @@ -20,7 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -32,10 +32,10 @@ public final class StructuralByteArrayTest { @Test public void testStructuralByteArray() throws Exception { assertEquals( - new StructuralByteArray("test string".getBytes(Charsets.UTF_8)), - new StructuralByteArray("test string".getBytes(Charsets.UTF_8))); + new StructuralByteArray("test string".getBytes(StandardCharsets.UTF_8)), + new StructuralByteArray("test string".getBytes(StandardCharsets.UTF_8))); assertFalse( - new StructuralByteArray("test string".getBytes(Charsets.UTF_8)) - .equals(new StructuralByteArray("diff string".getBytes(Charsets.UTF_8)))); + new StructuralByteArray("test string".getBytes(StandardCharsets.UTF_8)) + .equals(new StructuralByteArray("diff string".getBytes(StandardCharsets.UTF_8)))); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java index 7fd54039b1dd..c4f83954e66c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io; +import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.sdk.io.WriteFiles.UNKNOWN_SHARDNUM; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets.UTF_8; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java index b87c9caa1244..90b0822d9dca 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java @@ -31,6 +31,7 @@ import java.io.OutputStreamWriter; import java.io.Serializable; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.CopyOption; import java.nio.file.Files; import java.nio.file.Path; @@ -69,7 +70,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -368,10 +368,10 @@ public void testMatchWatchForNewFiles() throws IOException, InterruptedException public void testRead() throws IOException { final String path = tmpFolder.newFile("file").getAbsolutePath(); final String pathGZ = tmpFolder.newFile("file.gz").getAbsolutePath(); - Files.write(new File(path).toPath(), "Hello world".getBytes(Charsets.UTF_8)); + Files.write(new File(path).toPath(), "Hello world".getBytes(StandardCharsets.UTF_8)); try (Writer writer = new OutputStreamWriter( - new GZIPOutputStream(new FileOutputStream(pathGZ)), Charsets.UTF_8)) { + new GZIPOutputStream(new FileOutputStream(pathGZ)), StandardCharsets.UTF_8)) { writer.write("Hello world"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java index acde8c91431d..a38faf077e07 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java @@ -67,7 +67,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams; @@ -212,7 +211,7 @@ public void testWriteTwo() throws Exception { @Category(NeedsRunner.class) public void testReadInvalidRecord() throws Exception { expectedException.expectMessage("Not a valid TFRecord. Fewer than 12 bytes."); - runTestRead("bar".getBytes(Charsets.UTF_8), new String[0]); + runTestRead("bar".getBytes(StandardCharsets.UTF_8), new String[0]); } @Test @@ -445,14 +444,14 @@ private static Iterable makeLines(int n, int minRecordSize) { static class ByteArrayToString extends DoFn { @ProcessElement public void processElement(ProcessContext c) { - c.output(new String(c.element(), Charsets.UTF_8)); + c.output(new String(c.element(), StandardCharsets.UTF_8)); } } static class StringToByteArray extends DoFn { @ProcessElement public void processElement(ProcessContext c) { - c.output(c.element().getBytes(Charsets.UTF_8)); + c.output(c.element().getBytes(StandardCharsets.UTF_8)); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index 253308d1b93f..8d9adbefd02b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -51,6 +51,7 @@ import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -91,7 +92,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; 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; @@ -642,7 +642,7 @@ private void runTestRead(String[] expected) throws Exception { try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { for (String elem : expected) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -865,7 +865,7 @@ public void testProgressEmptyFile() throws IOException { public void testProgressTextFile() throws IOException { String file = "line1\nline2\nline3"; try (BoundedSource.BoundedReader reader = - prepareSource(file.getBytes(Charsets.UTF_8)) + prepareSource(file.getBytes(StandardCharsets.UTF_8)) .createReader(PipelineOptionsFactory.create())) { // Check preconditions before starting assertEquals(0.0, reader.getFractionConsumed(), 1e-6); @@ -901,7 +901,7 @@ public void testProgressTextFile() throws IOException { @Test public void testProgressAfterSplitting() throws IOException { String file = "line1\nline2\nline3"; - BoundedSource source = prepareSource(file.getBytes(Charsets.UTF_8)); + BoundedSource source = prepareSource(file.getBytes(StandardCharsets.UTF_8)); BoundedSource remainder; // Create the remainder, verifying properties pre- and post-splitting. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java index 312605f3fcc5..695ff4474d71 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java @@ -39,6 +39,7 @@ import java.io.OutputStream; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -73,7 +74,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Functions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; @@ -492,7 +492,7 @@ private static void assertOutputFiles( List expectedElements = new ArrayList<>(elems.length); for (String elem : elems) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); expectedElements.add(line); } @@ -509,7 +509,7 @@ private static void assertOutputFiles( private static List readLinesFromFile(File f) throws IOException { List currentFile = new ArrayList<>(); - try (BufferedReader reader = Files.newBufferedReader(f.toPath(), Charsets.UTF_8)) { + try (BufferedReader reader = Files.newBufferedReader(f.toPath(), StandardCharsets.UTF_8)) { while (true) { String line = reader.readLine(); if (line == null) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java index 17ca3ba85fd8..e52d4112e11e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextRowCountEstimatorTest.java @@ -20,8 +20,8 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.Writer; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Files; import org.junit.Assert; import org.junit.Rule; @@ -38,7 +38,7 @@ public class TextRowCountEstimatorTest { @Test public void testNonEmptyFiles() throws Exception { File file1 = temporaryFolder.newFile("file1.txt"); - Writer writer = Files.newWriter(file1, Charsets.UTF_8); + Writer writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 100; i++) { writer.write("123123123\n"); } @@ -47,7 +47,7 @@ public void testNonEmptyFiles() throws Exception { temporaryFolder.newFolder("testfolder"); temporaryFolder.newFolder("testfolder2"); file1 = temporaryFolder.newFile("testfolder/test2.txt"); - writer = Files.newWriter(file1, Charsets.UTF_8); + writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 50; i++) { writer.write("123123123\n"); } @@ -71,7 +71,7 @@ public void testEmptyFolder() throws Exception { @Test public void testEmptyFile() throws Exception { File file1 = temporaryFolder.newFile("file1.txt"); - Writer writer = Files.newWriter(file1, Charsets.UTF_8); + Writer writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 100; i++) { writer.write("\n"); } @@ -86,7 +86,7 @@ public void testEmptyFile() throws Exception { @Test(expected = TextRowCountEstimator.NoEstimationException.class) public void lotsOfNewLines() throws Exception { File file1 = temporaryFolder.newFile("file1.txt"); - Writer writer = Files.newWriter(file1, Charsets.UTF_8); + Writer writer = Files.newWriter(file1, StandardCharsets.UTF_8); for (int i = 0; i < 1000; i++) { writer.write("\n"); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 0ab8efac7eb1..cc174002bb46 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -35,6 +35,7 @@ import java.io.BufferedReader; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.text.DecimalFormat; import java.util.ArrayList; @@ -93,7 +94,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.ShardedKey; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -1035,7 +1035,8 @@ static void checkFileContents( List actual = Lists.newArrayList(); for (File outputFile : outputFiles) { List actualShard = Lists.newArrayList(); - try (BufferedReader reader = Files.newBufferedReader(outputFile.toPath(), Charsets.UTF_8)) { + try (BufferedReader reader = + Files.newBufferedReader(outputFile.toPath(), StandardCharsets.UTF_8)) { for (; ; ) { String line = reader.readLine(); if (line == null) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 2643fb556ff4..291bb5297880 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -59,6 +59,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.List; import java.util.Map; @@ -74,7 +75,6 @@ import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -1727,7 +1727,7 @@ public void testWhenNoHelpIsRequested() { assertFalse( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertEquals("", output); } @@ -1739,7 +1739,7 @@ public void testDefaultHelpAsArgument() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("The set of registered options are:")); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("Use --help= for detailed help.")); @@ -1753,7 +1753,7 @@ public void testSpecificHelpAsArgument() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); @@ -1769,7 +1769,7 @@ public void testSpecificHelpAsArgumentWithSimpleClassName() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); @@ -1785,7 +1785,7 @@ public void testSpecificHelpAsArgumentWithClassNameSuffix() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); @@ -1815,7 +1815,7 @@ public void testShortnameSpecificHelpHasMultipleMatches() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("Multiple matches found for NameConflict")); assertThat( output, @@ -1839,7 +1839,7 @@ public void testHelpWithOptionThatOutputsValidEnumTypes() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("")); } @@ -1851,7 +1851,7 @@ public void testHelpWithBadOptionNameAsArgument() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("Unable to find option org.apache.beam.sdk.Pipeline")); assertThat(output, containsString("The set of registered options are:")); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); @@ -1865,7 +1865,7 @@ public void testHelpWithHiddenMethodAndInterface() { assertTrue( PipelineOptionsFactory.printHelpUsageAndExitIfNeeded( arguments, new PrintStream(baos), false /* exit */)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); // A hidden interface. assertThat( output, not(containsString("org.apache.beam.sdk.options.DataflowPipelineDebugOptions"))); @@ -1877,7 +1877,7 @@ public void testHelpWithHiddenMethodAndInterface() { public void testProgrammaticPrintHelp() { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PipelineOptionsFactory.printHelp(new PrintStream(baos)); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("The set of registered options are:")); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); } @@ -1886,7 +1886,7 @@ public void testProgrammaticPrintHelp() { public void testProgrammaticPrintHelpForSpecificType() { ByteArrayOutputStream baos = new ByteArrayOutputStream(); PipelineOptionsFactory.printHelp(new PrintStream(baos), PipelineOptions.class); - String output = new String(baos.toByteArray(), Charsets.UTF_8); + String output = new String(baos.toByteArray(), StandardCharsets.UTF_8); assertThat(output, containsString("org.apache.beam.sdk.options.PipelineOptions")); assertThat(output, containsString("--runner")); assertThat(output, containsString("Default: " + DEFAULT_RUNNER_NAME)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java index bd7a0da394ae..3b22addbf545 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertThrows; import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.HashMap; @@ -54,7 +55,6 @@ import org.apache.beam.sdk.schemas.logicaltypes.VariableString; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Test; @@ -243,7 +243,8 @@ public static Iterable data() { .setUrn("pythonsdk:value") .setPayload( ByteString.copyFrom( - "some payload describing a python type", Charsets.UTF_8)) + "some payload describing a python type", + StandardCharsets.UTF_8)) .setRepresentation( SchemaApi.FieldType.newBuilder() .setAtomicType(SchemaApi.AtomicType.BYTES)) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java index 5298d29dad10..0c9e0065f5a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -29,12 +29,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Random; import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.collection.IsIterableContainingInOrder; import org.junit.Rule; @@ -165,7 +165,7 @@ public void testWritingByteWhenFinishedThrows() throws Exception { public void testWritingBytesWhenFinishedThrows() throws Exception { expectedException.expect(IOException.class); expectedException.expectMessage("Stream has been finished."); - testValues(toBytes("a")).write("b".getBytes(Charsets.UTF_8)); + testValues(toBytes("a")).write("b".getBytes(StandardCharsets.UTF_8)); } @Test @@ -203,7 +203,7 @@ public void testBehaviorWhenBufferPoolEmpty() throws Exception { private List toBytes(String... values) { ImmutableList.Builder builder = ImmutableList.builder(); for (String value : values) { - builder.add(value.getBytes(Charsets.UTF_8)); + builder.add(value.getBytes(StandardCharsets.UTF_8)); } return builder.build(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java index e87f6a2b0d0a..d26794274653 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java @@ -24,7 +24,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,7 +33,7 @@ @RunWith(JUnit4.class) public class ExposedByteArrayInputStreamTest { - private static final byte[] TEST_DATA = "Hello World!".getBytes(Charsets.UTF_8); + private static final byte[] TEST_DATA = "Hello World!".getBytes(StandardCharsets.UTF_8); private ByteArrayInputStream stream = new ByteArrayInputStream(TEST_DATA); @@ -74,6 +74,6 @@ public void testReadPartial() throws IOException { public void testReadAllAfterReadPartial() throws IOException { assertNotEquals(-1, exposedStream.read()); byte[] ret = exposedStream.readAll(); - assertArrayEquals("ello World!".getBytes(Charsets.UTF_8), ret); + assertArrayEquals("ello World!".getBytes(StandardCharsets.UTF_8), ret); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java index 7e1b213c85b2..a4a105a89ddc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java @@ -25,7 +25,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.UnsupportedEncodingException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -34,7 +34,7 @@ @RunWith(JUnit4.class) public class ExposedByteArrayOutputStreamTest { - private static final byte[] TEST_DATA = "Hello World!".getBytes(Charsets.UTF_8); + private static final byte[] TEST_DATA = "Hello World!".getBytes(StandardCharsets.UTF_8); private ExposedByteArrayOutputStream exposedStream = new ExposedByteArrayOutputStream(); private ByteArrayOutputStream stream = new ByteArrayOutputStream(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java index e15bd42dc3ce..1f3ec0f427b4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java @@ -24,12 +24,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.List; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.testing.InterceptingUrlClassLoader; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; @@ -93,7 +93,7 @@ public void testDeserializationError() { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("unable to deserialize a bogus string"); SerializableUtils.deserializeFromByteArray( - "this isn't legal".getBytes(Charsets.UTF_8), "a bogus string"); + "this isn't legal".getBytes(StandardCharsets.UTF_8), "a bogus string"); } /** A class that is not serializable by Java. */ diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java index 3bd87c2ae5c7..1c8d515d5c85 100644 --- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java +++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java @@ -33,6 +33,7 @@ import com.google.auto.value.AutoValue; import java.io.IOException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; @@ -60,7 +61,6 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.collect.Iterables; @@ -84,7 +84,7 @@ public class ExpansionServiceTest { private ExpansionService expansionService = new ExpansionService(); public static final List BYTE_LIST = ImmutableList.of("testing", "compound", "coders").stream() - .map(str -> str.getBytes(Charsets.UTF_8)) + .map(str -> str.getBytes(StandardCharsets.UTF_8)) .collect(Collectors.toList()); public static final Map BYTE_KV_LIST = ImmutableList.of("testing", "compound", "coders").stream() diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java index 67f386411d81..e56b95d7f8a6 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/AvroGenericCoderTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.avro; +import java.nio.charset.StandardCharsets; import java.util.Collections; import java.util.List; import org.apache.avro.Schema; @@ -24,7 +25,6 @@ import org.apache.beam.sdk.extensions.avro.coders.AvroGenericCoder; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.util.construction.CoderTranslator; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; /** Coder translator for AvroGenericCoder. */ public class AvroGenericCoderTranslator implements CoderTranslator { @@ -35,13 +35,13 @@ public List> getComponents(AvroGenericCoder from) { @Override public byte[] getPayload(AvroGenericCoder from) { - return from.getSchema().toString().getBytes(Charsets.UTF_8); + return from.getSchema().toString().getBytes(StandardCharsets.UTF_8); } @Override public AvroGenericCoder fromComponents( List> components, byte[] payload, TranslationContext context) { - Schema schema = new Schema.Parser().parse(new String(payload, Charsets.UTF_8)); + Schema schema = new Schema.Parser().parse(new String(payload, StandardCharsets.UTF_8)); return AvroGenericCoder.of(schema); } } diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java index 30a1a7787252..2a0bc36f6e9e 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java @@ -36,6 +36,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -100,7 +101,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -1436,7 +1436,7 @@ public void testMetadata() throws Exception { "longKey", 100L, "bytesKey", - "bytesValue".getBytes(Charsets.UTF_8)))); + "bytesValue".getBytes(StandardCharsets.UTF_8)))); writePipeline.run(); try (DataFileStream dataFileStream = @@ -1444,7 +1444,7 @@ public void testMetadata() throws Exception { assertEquals("stringValue", dataFileStream.getMetaString("stringKey")); assertEquals(100L, dataFileStream.getMetaLong("longKey")); assertArrayEquals( - "bytesValue".getBytes(Charsets.UTF_8), dataFileStream.getMeta("bytesKey")); + "bytesValue".getBytes(StandardCharsets.UTF_8), dataFileStream.getMeta("bytesKey")); } } diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index e4e5f3533445..c23a771f3cc8 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -61,7 +62,6 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -499,7 +499,8 @@ public OutputT expand(InputT input) { requirementsFile.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(requirementsFile.getAbsolutePath()), + StandardCharsets.UTF_8)) { for (String pkg : extraPackages) { fout.write(pkg); fout.write('\n'); diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java index 4392f23c4636..ab4d02ec838d 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonService.java @@ -24,13 +24,13 @@ import java.io.InputStreamReader; import java.net.ServerSocket; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeoutException; import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.io.ByteStreams; import org.slf4j.Logger; @@ -106,7 +106,8 @@ public AutoCloseable start() throws IOException, InterruptedException { new ProcessBuilder(bootstrapCommand).redirectError(ProcessBuilder.Redirect.INHERIT).start(); bootstrap.getOutputStream().close(); BufferedReader reader = - new BufferedReader(new InputStreamReader(bootstrap.getInputStream(), Charsets.UTF_8)); + new BufferedReader( + new InputStreamReader(bootstrap.getInputStream(), StandardCharsets.UTF_8)); String lastLine = reader.readLine(); String lastNonEmptyLine = lastLine; while (lastLine != null) { diff --git a/sdks/java/extensions/sql/jdbc/build.gradle b/sdks/java/extensions/sql/jdbc/build.gradle index 41fddce7116a..c5d462e0f5ca 100644 --- a/sdks/java/extensions/sql/jdbc/build.gradle +++ b/sdks/java/extensions/sql/jdbc/build.gradle @@ -35,11 +35,11 @@ dependencies { implementation "jline:jline:2.14.6" permitUnusedDeclared "jline:jline:2.14.6" // BEAM-11761 implementation "sqlline:sqlline:1.4.0" - implementation library.java.vendored_guava_32_1_2_jre implementation library.java.vendored_calcite_1_28_0 permitUnusedDeclared library.java.vendored_calcite_1_28_0 testImplementation project(path: ":sdks:java:io:google-cloud-platform", configuration: "testRuntimeMigration") testImplementation library.java.junit + testImplementation library.java.vendored_guava_32_1_2_jre // Depending on outputs so integrationTest can run with only test dependencies. // This enables us to test the JDBC jar being loaded on a custom classloader. integrationTest sourceSets.test.output diff --git a/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java b/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java index ac049608ebcb..8c87343cd7c1 100644 --- a/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java +++ b/sdks/java/extensions/sql/jdbc/src/main/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLine.java @@ -23,10 +23,10 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.PrintStream; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.checkerframework.checker.nullness.qual.Nullable; import sqlline.SqlLine; import sqlline.SqlLine.Status; @@ -68,11 +68,11 @@ static Status runSqlLine( SqlLine sqlLine = new SqlLine(); if (outputStream != null) { - sqlLine.setOutputStream(new PrintStream(outputStream, false, Charsets.UTF_8.name())); + sqlLine.setOutputStream(new PrintStream(outputStream, false, StandardCharsets.UTF_8.name())); } if (errorStream != null) { - sqlLine.setErrorStream(new PrintStream(errorStream, false, Charsets.UTF_8.name())); + sqlLine.setErrorStream(new PrintStream(errorStream, false, StandardCharsets.UTF_8.name())); } return sqlLine.begin(modifiedArgs, inputStream, true); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java index e5a46f877001..e34106db1d93 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import java.io.File; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import org.apache.beam.sdk.extensions.sql.SqlTransform; import org.apache.beam.sdk.schemas.Schema; @@ -33,7 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -81,7 +81,7 @@ protected void after() {} public void testLegacyDefaultCsv() throws Exception { Files.write( tempFolder.newFile("test.csv").toPath(), - "hello,13\n\ngoodbye,42\n".getBytes(Charsets.UTF_8)); + "hello,13\n\ngoodbye,42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = String.format( @@ -105,7 +105,7 @@ public void testLegacyDefaultCsv() throws Exception { public void testLegacyTdfCsv() throws Exception { Files.write( tempFolder.newFile("test.csv").toPath(), - "hello\t13\n\ngoodbye\t42\n".getBytes(Charsets.UTF_8)); + "hello\t13\n\ngoodbye\t42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -138,7 +138,7 @@ public void testLegacyTdfCsv() throws Exception { public void testExplicitCsv() throws Exception { Files.write( tempFolder.newFile("test.csv").toPath(), - "hello,13\n\ngoodbye,42\n".getBytes(Charsets.UTF_8)); + "hello,13\n\ngoodbye,42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -165,7 +165,8 @@ public void testExplicitCsv() throws Exception { @Test public void testExplicitCsvExcel() throws Exception { Files.write( - tempFolder.newFile("test.csv").toPath(), "hello\n\ngoodbye\n".getBytes(Charsets.UTF_8)); + tempFolder.newFile("test.csv").toPath(), + "hello\n\ngoodbye\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -191,7 +192,8 @@ public void testExplicitCsvExcel() throws Exception { public void testLines() throws Exception { // Data that looks like CSV but isn't parsed as it Files.write( - tempFolder.newFile("test.csv").toPath(), "hello,13\ngoodbye,42\n".getBytes(Charsets.UTF_8)); + tempFolder.newFile("test.csv").toPath(), + "hello,13\ngoodbye,42\n".getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -210,7 +212,8 @@ public void testLines() throws Exception { @Test public void testJson() throws Exception { - Files.write(tempFolder.newFile("test.json").toPath(), JSON_TEXT.getBytes(Charsets.UTF_8)); + Files.write( + tempFolder.newFile("test.json").toPath(), JSON_TEXT.getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = @@ -229,7 +232,8 @@ public void testJson() throws Exception { public void testInvalidJson() throws Exception { File deadLetterFile = new File(tempFolder.getRoot(), "dead-letter-file"); Files.write( - tempFolder.newFile("test.json").toPath(), INVALID_JSON_TEXT.getBytes(Charsets.UTF_8)); + tempFolder.newFile("test.json").toPath(), + INVALID_JSON_TEXT.getBytes(StandardCharsets.UTF_8)); String query = "SELECT * FROM test"; String ddl = diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java index c8c49a656e3b..09a6ced44d37 100644 --- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java +++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java @@ -21,12 +21,12 @@ import com.clickhouse.client.ClickHousePipedOutputStream; import com.clickhouse.client.data.BinaryStreamUtils; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.clickhouse.TableSchema.ColumnType; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.RowWithStorage; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.joda.time.Days; import org.joda.time.Instant; @@ -60,7 +60,7 @@ static void writeValue(ClickHouseOutputStream stream, ColumnType columnType, Obj byte[] bytes; if (value instanceof String) { - bytes = ((String) value).getBytes(Charsets.UTF_8); + bytes = ((String) value).getBytes(StandardCharsets.UTF_8); } else { bytes = ((byte[]) value); } diff --git a/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java b/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java index 48904cae430f..2cc89a2a0dc1 100644 --- a/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java +++ b/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java @@ -44,6 +44,7 @@ import java.io.OutputStream; import java.io.PrintStream; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -87,7 +88,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; 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; @@ -524,7 +524,7 @@ public String createFileFromList(List input) throws Exception { try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { for (String elem : input) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -548,7 +548,7 @@ public void multipleFilesTest() throws Exception { for (int lineNum = 0; lineNum < numLines; ++lineNum) { String elem = filename + " " + lineNum; byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -588,7 +588,7 @@ public void testWithHintMatchesManyFiles() throws IOException { for (int lineNum = 0; lineNum < 10 + num; ++lineNum) { String elem = filename + " " + lineNum; byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -817,7 +817,7 @@ private void runTestRead(String[] expected) throws Exception { try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { for (String elem : expected) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); } } @@ -840,7 +840,7 @@ private void runTestReadLineNumsAndFileName(String[] expected) throws Exception int lineNum = 0; for (String elem : expected) { byte[] encodedElem = CoderUtils.encodeToByteArray(StringUtf8Coder.of(), elem); - String line = new String(encodedElem, Charsets.UTF_8); + String line = new String(encodedElem, StandardCharsets.UTF_8); writer.println(line); actualExpected.add(lineNum + " " + filePath + " " + line); lineNum++; @@ -1121,7 +1121,7 @@ public void testProgressEmptyFile() throws IOException { public void testProgressTextFile() throws IOException { String file = "line1\nline2\nline3"; try (BoundedSource.BoundedReader reader = - prepareSource(file.getBytes(Charsets.UTF_8)) + prepareSource(file.getBytes(StandardCharsets.UTF_8)) .createReader(PipelineOptionsFactory.create())) { // Check preconditions before starting assertEquals(0.0, reader.getFractionConsumed(), 1e-6); @@ -1157,7 +1157,7 @@ public void testProgressTextFile() throws IOException { @Test public void testProgressAfterSplitting() throws IOException { String file = "line1\nline2\nline3"; - BoundedSource source = prepareSource(file.getBytes(Charsets.UTF_8)); + BoundedSource source = prepareSource(file.getBytes(StandardCharsets.UTF_8)); BoundedSource remainder; // Create the remainder, verifying properties pre- and post-splitting. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java index dd5a9abd5ac8..98aade888a33 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubReadSchemaTransformProviderTest.java @@ -25,6 +25,7 @@ import com.google.protobuf.Timestamp; import java.io.IOException; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -47,7 +48,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.junit.Rule; @@ -170,7 +170,7 @@ public void testReadRaw() throws IOException { PCollectionRowTuple begin = PCollectionRowTuple.empty(p); Schema rawSchema = Schema.of(Schema.Field.of("payload", Schema.FieldType.BYTES)); - byte[] payload = "some payload".getBytes(Charsets.UTF_8); + byte[] payload = "some payload".getBytes(StandardCharsets.UTF_8); try (PubsubTestClientFactory clientFactory = clientFactory(ImmutableList.of(incomingMessageOf(payload, CLOCK.currentTimeMillis())))) { @@ -211,7 +211,7 @@ public void testReadAttributes() throws IOException { .addStringField("attr") .addMapField("attrMap", Schema.FieldType.STRING, Schema.FieldType.STRING) .build(); - byte[] payload = "some payload".getBytes(Charsets.UTF_8); + byte[] payload = "some payload".getBytes(StandardCharsets.UTF_8); String attr = "attr value"; try (PubsubTestClientFactory clientFactory = diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java index 612b20393d78..6ee3d9d96ef6 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -57,7 +58,6 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.collect.Iterables; @@ -257,8 +257,8 @@ public synchronized ConsumerRecords poll(Duration timeout) { topicPartition.topic(), topicPartition.partition(), startOffset + i, - key.getBytes(Charsets.UTF_8), - value.getBytes(Charsets.UTF_8))); + key.getBytes(StandardCharsets.UTF_8), + value.getBytes(StandardCharsets.UTF_8))); } if (records.isEmpty()) { return ConsumerRecords.empty(); diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java index 4983d52a642f..8071bad84d73 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.tpcds; +import java.nio.charset.StandardCharsets; import java.util.Set; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParseException; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; /** @@ -39,7 +39,7 @@ public class QueryReader { */ public static String readQuery(String queryFileName) throws Exception { String path = "queries/" + queryFileName + ".sql"; - return Resources.toString(Resources.getResource(path), Charsets.UTF_8); + return Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8); } /** diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java index 1550a25b7c8f..6efb7e7e0659 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java @@ -21,6 +21,7 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -56,7 +57,6 @@ import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.util.SqlBasicVisitor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; 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.io.Resources; import org.apache.commons.csv.CSVFormat; @@ -212,7 +212,7 @@ private static PCollection getTableCSV( private static org.apache.avro.Schema getAvroSchema(String tableName) throws IOException { String path = "schemas_avro/" + tableName + ".json"; return new org.apache.avro.Schema.Parser() - .parse(Resources.toString(Resources.getResource(path), Charsets.UTF_8)); + .parse(Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8)); } static org.apache.avro.Schema getProjectedSchema( diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java index 485fa83a4a8e..97116e14cdcd 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TableSchemaJSONLoader.java @@ -20,11 +20,11 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Resources; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.reflect.ClassPath; import org.json.simple.JSONArray; @@ -49,7 +49,7 @@ public class TableSchemaJSONLoader { @SuppressWarnings({"rawtypes", "DefaultCharset"}) public static String parseTableSchema(String tableName) throws Exception { String path = "schemas/" + tableName + ".json"; - String schema = Resources.toString(Resources.getResource(path), Charsets.UTF_8); + String schema = Resources.toString(Resources.getResource(path), StandardCharsets.UTF_8); JSONObject jsonObject = (JSONObject) new JSONParser().parse(schema); JSONArray jsonArray = (JSONArray) jsonObject.get("schema"); diff --git a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java index a9ae5360a859..b766d2b13a4b 100644 --- a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java +++ b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java @@ -25,12 +25,12 @@ import java.io.IOException; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; import java.util.UUID; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -77,7 +77,7 @@ public void testLauncherInstallsDependencies() throws IOException { try (Writer fout = new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(requirementsFile.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write("pypipackage1\n"); fout.write("pypipackage2\n"); } @@ -118,7 +118,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { dependency1.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(dependency1.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(dependency1.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write("tempdata\n"); } @@ -128,7 +128,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { dependency2.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(dependency2.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(dependency2.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write("tempdata\n"); } @@ -140,7 +140,7 @@ public void testLauncherInstallsLocalDependencies() throws IOException { requirementsFile.deleteOnExit(); try (Writer fout = new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + new FileOutputStream(requirementsFile.getAbsolutePath()), StandardCharsets.UTF_8)) { fout.write(dependency1.getAbsolutePath() + "\n"); fout.write(dependency2.getAbsolutePath() + "\n"); fout.write("pypipackage" + "\n"); From 99a23830037f58178d3fdf9db22f27b4de37dac4 Mon Sep 17 00:00:00 2001 From: Damon Date: Tue, 6 Aug 2024 09:48:47 -0700 Subject: [PATCH 09/21] Enable artifact staging during Prism Runner lifecycle (#32084) --- runners/prism/java/build.gradle | 1 + .../runners/prism/PrismArtifactStager.java | 173 ++++++++++++++++++ .../prism/PrismArtifactStagerTest.java | 143 +++++++++++++++ 3 files changed, 317 insertions(+) create mode 100644 runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java create mode 100644 runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java diff --git a/runners/prism/java/build.gradle b/runners/prism/java/build.gradle index 2b0635ca6125..96ab4e70a579 100644 --- a/runners/prism/java/build.gradle +++ b/runners/prism/java/build.gradle @@ -30,6 +30,7 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:harness", configuration: "shadow") + implementation project(":runners:java-fn-execution") implementation project(":runners:portability:java") implementation library.java.joda_time diff --git a/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java new file mode 100644 index 000000000000..f1d99a213eea --- /dev/null +++ b/runners/prism/java/src/main/java/org/apache/beam/runners/prism/PrismArtifactStager.java @@ -0,0 +1,173 @@ +/* + * 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.prism; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc; +import org.apache.beam.model.jobmanagement.v1.JobApi; +import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc; +import org.apache.beam.model.pipeline.v1.Endpoints; +import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; +import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; +import org.apache.beam.sdk.fn.channel.ManagedChannelFactory; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Stages {@link org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline} artifacts of prepared jobs. + */ +@AutoValue +abstract class PrismArtifactStager implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(PrismArtifactStager.class); + + /** + * Instantiate a {@link PrismArtifactStager} via call to {@link #of(String, String)}, assigning + * {@link Builder#setStagingEndpoint} using {@param prepareJobResponse} {@link + * JobApi.PrepareJobResponse#getArtifactStagingEndpoint} and {@link + * JobApi.PrepareJobResponse#getStagingSessionToken}. + */ + static PrismArtifactStager of(JobApi.PrepareJobResponse prepareJobResponse) { + return of( + prepareJobResponse.getArtifactStagingEndpoint().getUrl(), + prepareJobResponse.getStagingSessionToken()); + } + + /** + * Instantiates a {@link PrismArtifactStager} from the {@param stagingEndpoint} URL and {@param + * stagingSessionToken} to instantiate the {@link #getRetrievalService}, {@link + * #getManagedChannel}, and {@link #getStagingServiceStub} defaults. See the referenced getters + * for more details. + */ + static PrismArtifactStager of(String stagingEndpoint, String stagingSessionToken) { + return PrismArtifactStager.builder() + .setStagingEndpoint(stagingEndpoint) + .setStagingSessionToken(stagingSessionToken) + .build(); + } + + static Builder builder() { + return new AutoValue_PrismArtifactStager.Builder(); + } + + /** + * Stage the {@link org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline} artifacts via {@link + * ArtifactStagingService#offer} supplying {@link #getRetrievalService}, {@link + * #getStagingServiceStub}, and {@link #getStagingSessionToken}. + */ + void stage() throws ExecutionException, InterruptedException { + LOG.info("staging artifacts at {}", getStagingEndpoint()); + ArtifactStagingService.offer( + getRetrievalService(), getStagingServiceStub(), getStagingSessionToken()); + } + + /** The URL of the {@link ArtifactStagingService}. */ + abstract String getStagingEndpoint(); + + /** + * Token associated with a staging session and acquired from a {@link + * JobServiceGrpc.JobServiceStub#prepare}'s {@link JobApi.PrepareJobResponse}. + */ + abstract String getStagingSessionToken(); + + /** + * The service that retrieves artifacts; defaults to instantiating from the default {@link + * ArtifactRetrievalService#ArtifactRetrievalService()} constructor. + */ + abstract ArtifactRetrievalService getRetrievalService(); + + /** + * Used to instantiate the {@link #getStagingServiceStub}. By default, instantiates using {@link + * ManagedChannelFactory#forDescriptor(Endpoints.ApiServiceDescriptor)}, where {@link + * Endpoints.ApiServiceDescriptor} is instantiated via {@link + * Endpoints.ApiServiceDescriptor.Builder#setUrl(String)} and the URL provided by {@link + * #getStagingEndpoint}. + */ + abstract ManagedChannel getManagedChannel(); + + /** + * Required by {@link ArtifactStagingService#offer}. By default, instantiates using {@link + * ArtifactStagingServiceGrpc#newStub} and {@link #getManagedChannel}. + */ + abstract ArtifactStagingServiceGrpc.ArtifactStagingServiceStub getStagingServiceStub(); + + @Override + public void close() { + LOG.info("shutting down {}", PrismArtifactStager.class); + getRetrievalService().close(); + getManagedChannel().shutdown(); + try { + getManagedChannel().awaitTermination(3000L, TimeUnit.MILLISECONDS); + } catch (InterruptedException ignored) { + } + } + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setStagingEndpoint(String stagingEndpoint); + + abstract Optional getStagingEndpoint(); + + abstract Builder setStagingSessionToken(String stagingSessionToken); + + abstract Builder setRetrievalService(ArtifactRetrievalService retrievalService); + + abstract Optional getRetrievalService(); + + abstract Builder setManagedChannel(ManagedChannel managedChannel); + + abstract Optional getManagedChannel(); + + abstract Builder setStagingServiceStub( + ArtifactStagingServiceGrpc.ArtifactStagingServiceStub stub); + + abstract Optional + getStagingServiceStub(); + + abstract PrismArtifactStager autoBuild(); + + final PrismArtifactStager build() { + + checkState(getStagingEndpoint().isPresent(), "missing staging endpoint"); + ManagedChannelFactory channelFactory = ManagedChannelFactory.createDefault(); + + if (!getManagedChannel().isPresent()) { + Endpoints.ApiServiceDescriptor descriptor = + Endpoints.ApiServiceDescriptor.newBuilder().setUrl(getStagingEndpoint().get()).build(); + setManagedChannel(channelFactory.forDescriptor(descriptor)); + } + + if (!getStagingServiceStub().isPresent()) { + setStagingServiceStub(ArtifactStagingServiceGrpc.newStub(getManagedChannel().get())); + } + + if (!getRetrievalService().isPresent()) { + setRetrievalService(new ArtifactRetrievalService()); + } + + return autoBuild(); + } + } +} diff --git a/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java new file mode 100644 index 000000000000..d3ac8a72eafb --- /dev/null +++ b/runners/prism/java/src/test/java/org/apache/beam/runners/prism/PrismArtifactStagerTest.java @@ -0,0 +1,143 @@ +/* + * 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.prism; + +import static com.google.common.truth.Truth.assertThat; +import static org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService.EMBEDDED_ARTIFACT_URN; +import static org.junit.Assert.assertThrows; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService; +import org.apache.beam.runners.fnexecution.artifact.ArtifactStagingService; +import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessChannelBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder; +import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.commons.io.output.ByteArrayOutputStream; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link PrismArtifactStager}. */ +@RunWith(JUnit4.class) +public class PrismArtifactStagerTest { + + @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule(); + + final ArtifactStagingService stagingService = + new ArtifactStagingService(new TestDestinationProvider()); + + @Test + public void givenValidArtifacts_stages() + throws IOException, ExecutionException, InterruptedException { + PrismArtifactStager underTest = prismArtifactStager(validArtifacts()); + assertThat(underTest.getManagedChannel().isShutdown()).isFalse(); + underTest.stage(); + assertThat(stagingService.getStagedArtifacts(underTest.getStagingSessionToken())).isNotEmpty(); + underTest.close(); + assertThat(underTest.getManagedChannel().isShutdown()).isTrue(); + } + + @Test + public void givenErrors_performsGracefulCleanup() throws IOException { + PrismArtifactStager underTest = prismArtifactStager(invalidArtifacts()); + assertThat(underTest.getManagedChannel().isShutdown()).isFalse(); + ExecutionException error = assertThrows(ExecutionException.class, underTest::stage); + assertThat(error.getMessage()).contains("Unexpected artifact type: invalid-type-urn"); + assertThat(underTest.getManagedChannel().isShutdown()).isFalse(); + underTest.close(); + assertThat(underTest.getManagedChannel().isShutdown()).isTrue(); + } + + private PrismArtifactStager prismArtifactStager( + Map> artifacts) throws IOException { + String serverName = InProcessServerBuilder.generateName(); + ArtifactRetrievalService retrievalService = new ArtifactRetrievalService(); + String stagingToken = "staging-token"; + stagingService.registerJob(stagingToken, artifacts); + + grpcCleanup.register( + InProcessServerBuilder.forName(serverName) + .directExecutor() + .addService(stagingService) + .addService(retrievalService) + .build() + .start()); + + ManagedChannel channel = + grpcCleanup.register(InProcessChannelBuilder.forName(serverName).build()); + + return PrismArtifactStager.builder() + .setStagingEndpoint("ignore") + .setStagingSessionToken(stagingToken) + .setManagedChannel(channel) + .build(); + } + + private Map> validArtifacts() { + return ImmutableMap.of( + "env1", + Collections.singletonList( + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn(EMBEDDED_ARTIFACT_URN) + .setTypePayload( + RunnerApi.EmbeddedFilePayload.newBuilder() + .setData(ByteString.copyFromUtf8("type-payload")) + .build() + .toByteString()) + .setRoleUrn("role-urn") + .build())); + } + + private Map> invalidArtifacts() { + return ImmutableMap.of( + "env1", + Collections.singletonList( + RunnerApi.ArtifactInformation.newBuilder() + .setTypeUrn("invalid-type-urn") + .setTypePayload( + RunnerApi.EmbeddedFilePayload.newBuilder() + .setData(ByteString.copyFromUtf8("type-payload")) + .build() + .toByteString()) + .setRoleUrn("role-urn") + .build())); + } + + private static class TestDestinationProvider + implements ArtifactStagingService.ArtifactDestinationProvider { + + @Override + public ArtifactStagingService.ArtifactDestination getDestination( + String stagingToken, String name) throws IOException { + return ArtifactStagingService.ArtifactDestination.create( + EMBEDDED_ARTIFACT_URN, ByteString.EMPTY, new ByteArrayOutputStream()); + } + + @Override + public void removeStagedArtifacts(String stagingToken) throws IOException {} + } +} From 741facf00993f24cca9418078ac62ff53c28e04e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 11:18:56 -0700 Subject: [PATCH 10/21] Bump github.com/docker/docker in /sdks (#32046) Bumps [github.com/docker/docker](https://github.com/docker/docker) from 25.0.5+incompatible to 25.0.6+incompatible. - [Release notes](https://github.com/docker/docker/releases) - [Commits](https://github.com/docker/docker/compare/v25.0.5...v25.0.6) --- updated-dependencies: - dependency-name: github.com/docker/docker dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 6d42e02296c7..1716a6e2d22d 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -137,7 +137,7 @@ require ( github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect github.com/containerd/containerd v1.7.11 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect - github.com/docker/docker v25.0.5+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v25.0.6+incompatible // 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.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 098f858488b7..a0b4738decc3 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -768,8 +768,8 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/docker/docker v25.0.5+incompatible h1:UmQydMduGkrD5nQde1mecF/YnSbTOaPeFIeP5C4W+DE= -github.com/docker/docker v25.0.5+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v25.0.6+incompatible h1:5cPwbwriIcsua2REJe8HqQV+6WlWc1byg2QSXzBxBGg= +github.com/docker/docker v25.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= From 5ab908b984d4144b5cbe584d7ed4ed7a4e226993 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Aug 2024 15:03:57 -0400 Subject: [PATCH 11/21] Add Lineage metrics for BigtableIO (#32068) * Add Lineage metrics for BigtableIO * add tests * simplify metrics query logics; exclude test actually already failing * Address comments, fix typo --- .../org/apache/beam/sdk/metrics/Lineage.java | 43 ++++++++++++++++--- .../io/google-cloud-platform/build.gradle | 4 ++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 14 ++++++ .../sdk/io/gcp/bigtable/BigtableService.java | 6 +++ .../io/gcp/bigtable/BigtableServiceImpl.java | 22 ++++++++++ .../io/gcp/bigquery/BigQueryIOReadTest.java | 23 ++-------- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 13 +----- .../sdk/io/gcp/bigtable/BigtableReadIT.java | 21 ++++++++- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 18 +++++++- 9 files changed, 123 insertions(+), 41 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java index 7890a9f74b94..8b69b0ef5523 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Lineage.java @@ -17,17 +17,17 @@ */ package org.apache.beam.sdk.metrics; +import java.util.HashSet; +import java.util.Set; + /** * Standard collection of metrics used to record source and sinks information for lineage tracking. */ public class Lineage { - public static final String LINEAGE_NAMESPACE = "lineage"; - public static final String SOURCE_METRIC_NAME = "sources"; - public static final String SINK_METRIC_NAME = "sinks"; - - private static final StringSet SOURCES = Metrics.stringSet(LINEAGE_NAMESPACE, SOURCE_METRIC_NAME); - private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, SINK_METRIC_NAME); + private static final StringSet SOURCES = + Metrics.stringSet(LINEAGE_NAMESPACE, Type.SOURCE.toString()); + private static final StringSet SINKS = Metrics.stringSet(LINEAGE_NAMESPACE, Type.SINK.toString()); /** {@link StringSet} representing sources and optionally side inputs. */ public static StringSet getSources() { @@ -38,4 +38,35 @@ public static StringSet getSources() { public static StringSet getSinks() { return SINKS; } + + /** Query {@link StringSet} metrics from {@link MetricResults}. */ + public static Set query(MetricResults results, Type type) { + MetricsFilter filter = + MetricsFilter.builder() + .addNameFilter(MetricNameFilter.named(LINEAGE_NAMESPACE, type.toString())) + .build(); + Set result = new HashSet<>(); + for (MetricResult metrics : results.queryMetrics(filter).getStringSets()) { + result.addAll(metrics.getCommitted().getStringSet()); + result.addAll(metrics.getAttempted().getStringSet()); + } + return result; + } + + /** Lineage metrics resource types. */ + public enum Type { + SOURCE("source"), + SINK("sink"); + + private final String name; + + Type(String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + } } diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index e499bae6fc64..23c56f13a94c 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -218,6 +218,10 @@ task integrationTest(type: Test, dependsOn: processTestResources) { useJUnit { excludeCategories "org.apache.beam.sdk.testing.UsesKms" + filter { + // https://github.com/apache/beam/issues/32071 + excludeTestsMatching 'org.apache.beam.sdk.io.gcp.bigtable.BigtableReadIT.testE2EBigtableSegmentRead' + } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index d78ae2cb6c57..6d20109e947b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1337,6 +1337,7 @@ private static class BigtableWriterFn private transient Set> badRecords = null; // Due to callback thread not supporting Beam metrics, Record pending metrics and report later. private transient long pendingThrottlingMsecs; + private transient boolean reportedLineage; // Assign serviceEntry in startBundle and clear it in tearDown. @Nullable private BigtableServiceEntry serviceEntry; @@ -1480,6 +1481,10 @@ public void finishBundle(FinishBundleContext c) throws Exception { throttlingMsecs.inc(excessTime); } } + if (!reportedLineage) { + bigtableWriter.reportLineage(); + reportedLineage = true; + } bigtableWriter = null; } @@ -1612,6 +1617,7 @@ public String toString() { private final BigtableConfig config; private final BigtableReadOptions readOptions; private @Nullable Long estimatedSizeBytes; + private transient boolean reportedLineage; private final BigtableServiceFactory.ConfigId configId; @@ -1989,6 +1995,13 @@ public List getRanges() { public ValueProvider getTableId() { return readOptions.getTableId(); } + + void reportLineageOnce(BigtableService.Reader reader) { + if (!reportedLineage) { + reader.reportLineage(); + reportedLineage = true; + } + } } private static class BigtableReader extends BoundedReader { @@ -2019,6 +2032,7 @@ true, makeByteKey(reader.getCurrentRow().getKey()))) || rangeTracker.markDone(); if (hasRecord) { ++recordsReturned; + source.reportLineageOnce(reader); } return hasRecord; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 261cc3ac081d..50d8126999c4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -57,6 +57,9 @@ CompletionStage writeRecord(KV * @throws IOException if there is an error closing the writer */ void close() throws IOException; + + /** Report Lineage metrics to runner. */ + default void reportLineage() {} } /** The interface of a class that reads from Cloud Bigtable. */ @@ -77,6 +80,9 @@ interface Reader { Row getCurrentRow() throws NoSuchElementException; void close(); + + /** Report Lineage metrics to runner. */ + default void reportLineage() {} } /** Returns a {@link Reader} that will read from the specified source. */ diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index f06a4a127686..6fdf67722bac 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -71,6 +71,7 @@ import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; import org.apache.beam.sdk.io.range.ByteKeyRange; import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -212,6 +213,11 @@ public void close() { exhausted = true; } } + + @Override + public void reportLineage() { + Lineage.getSources().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); + } } @VisibleForTesting @@ -225,6 +231,9 @@ static class BigtableSegmentReaderImpl implements Reader { private final int refillSegmentWaterMark; private final long maxSegmentByteSize; private ServiceCallMetric serviceCallMetric; + private final String projectId; + private final String instanceId; + private final String tableId; private static class UpstreamResults { private final List rows; @@ -308,11 +317,19 @@ static BigtableSegmentReaderImpl create( // Asynchronously refill buffer when there is 10% of the elements are left this.refillSegmentWaterMark = Math.max(1, (int) (request.getRowsLimit() * WATERMARK_PERCENTAGE)); + this.projectId = projectId; + this.instanceId = instanceId; + this.tableId = tableId; } @Override public void close() {} + @Override + public void reportLineage() { + Lineage.getSources().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); + } + @Override public boolean start() throws IOException { future = fetchNextSegment(); @@ -578,6 +595,11 @@ public void writeSingleRecord(KV> record) throws } } + @Override + public void reportLineage() { + Lineage.getSinks().add(String.format("bigtable:%s.%s.%s", projectId, instanceId, tableId)); + } + private ServiceCallMetric createServiceCallMetric() { // Populate metrics HashMap baseLabels = new HashMap<>(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java index 5c43666e79e5..a8aca7570b33 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java @@ -43,6 +43,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ExecutionException; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificRecordBase; @@ -61,9 +62,6 @@ import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; import org.apache.beam.sdk.io.gcp.testing.FakeJobService; import org.apache.beam.sdk.metrics.Lineage; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; @@ -351,18 +349,8 @@ private void checkTypedReadQueryObjectWithValidate( } private void checkLineageSourceMetric(PipelineResult pipelineResult, String tableName) { - MetricQueryResults lineageMetrics = - pipelineResult - .metrics() - .queryMetrics( - MetricsFilter.builder() - .addNameFilter( - MetricNameFilter.named( - Lineage.LINEAGE_NAMESPACE, Lineage.SOURCE_METRIC_NAME)) - .build()); - assertThat( - lineageMetrics.getStringSets().iterator().next().getCommitted().getStringSet(), - contains("bigquery:" + tableName.replace(':', '.'))); + Set result = Lineage.query(pipelineResult.metrics(), Lineage.Type.SOURCE); + assertThat(result, contains("bigquery:" + tableName.replace(':', '.'))); } @Before @@ -600,10 +588,7 @@ public void processElement(ProcessContext c) throws Exception { new MyData("b", 2L, bd1, bd2), new MyData("c", 3L, bd1, bd2))); PipelineResult result = p.run(); - // Skip when direct runner splits outside of a counters context. - if (useTemplateCompatibility) { - checkLineageSourceMetric(result, "non-executing-project:somedataset.sometable"); - } + checkLineageSourceMetric(result, "non-executing-project:somedataset.sometable"); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index bc90d4c8bae7..c5af8045bfe2 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -118,9 +118,6 @@ import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService; import org.apache.beam.sdk.io.gcp.testing.FakeJobService; import org.apache.beam.sdk.metrics.Lineage; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricsFilter; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.schemas.JavaFieldSchema; @@ -285,16 +282,8 @@ public void evaluate() throws Throwable { .withJobService(fakeJobService); private void checkLineageSinkMetric(PipelineResult pipelineResult, String tableName) { - MetricQueryResults lineageMetrics = - pipelineResult - .metrics() - .queryMetrics( - MetricsFilter.builder() - .addNameFilter( - MetricNameFilter.named(Lineage.LINEAGE_NAMESPACE, Lineage.SINK_METRIC_NAME)) - .build()); assertThat( - lineageMetrics.getStringSets().iterator().next().getCommitted().getStringSet(), + Lineage.query(pipelineResult.metrics(), Lineage.Type.SINK), hasItem("bigquery:" + tableName.replace(':', '.'))); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java index bc88858ebc33..4ce9ad10b2c0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; + import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; @@ -28,7 +31,9 @@ import java.util.Date; import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -110,7 +115,8 @@ public void testE2EBigtableRead() { p.apply(BigtableIO.read().withBigtableOptions(bigtableOptionsBuilder).withTableId(tableId)) .apply(Count.globally()); PAssert.thatSingleton(count).isEqualTo(numRows); - p.run(); + PipelineResult r = p.run(); + checkLineageSourceMetric(r, tableId); } @Test @@ -138,6 +144,17 @@ public void testE2EBigtableSegmentRead() { .withMaxBufferElementCount(10)) .apply(Count.globally()); PAssert.thatSingleton(count).isEqualTo(numRows); - p.run(); + PipelineResult r = p.run(); + checkLineageSourceMetric(r, tableId); + } + + private void checkLineageSourceMetric(PipelineResult r, String tableId) { + // TODO(https://github.com/apache/beam/issues/32071) test malformed, + // when pipeline.run() is non-blocking, the metrics are not available by the time of query + if (options.getRunner().getName().contains("DirectRunner")) { + assertThat( + Lineage.query(r.metrics(), Lineage.Type.SOURCE), + hasItem(String.format("bigtable:%s.%s.%s", project, options.getInstanceId(), tableId))); + } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index bf9f7d991fa2..46bb3df836e5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigtable; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import com.google.api.gax.rpc.ServerStream; @@ -39,8 +40,10 @@ import java.util.Objects; import java.util.stream.Collectors; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.testing.PAssert; @@ -142,7 +145,7 @@ public void processElement(ProcessContext c) { .withProjectId(project) .withInstanceId(options.getInstanceId()) .withTableId(tableId)); - p.run(); + PipelineResult r = p.run(); // Test number of column families and column family name equality Table table = getTable(tableId); @@ -154,6 +157,7 @@ public void processElement(ProcessContext c) { // Test table data equality List> tableData = getTableData(tableId); assertThat(tableData, Matchers.containsInAnyOrder(testData.toArray())); + checkLineageSinkMetric(r, tableId); } @Test @@ -340,7 +344,7 @@ public void failureTest(int numRows, DoFn> tableData = getTableData(tableId); assertEquals(998, tableData.size()); + checkLineageSinkMetric(r, tableId); } @After @@ -412,4 +417,13 @@ private void deleteTable(String tableId) { tableAdminClient.deleteTable(tableId); } } + + private void checkLineageSinkMetric(PipelineResult r, String tableId) { + // Only check lineage metrics on direct runner until Dataflow runner v2 supported report back + if (options.getRunner().getName().contains("DirectRunner")) { + assertThat( + Lineage.query(r.metrics(), Lineage.Type.SINK), + hasItem(String.format("bigtable:%s.%s.%s", project, options.getInstanceId(), tableId))); + } + } } From 17283bb8294f22edfc4d00c49bf3d9a518a1551b Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Aug 2024 15:35:16 -0400 Subject: [PATCH 12/21] Add Lineage metrics to PubsubIO (#32037) * Add Lineage metrics to PubsubIO * fix format and add test * make getDatacatalogname fail safe --- .../beam/sdk/io/gcp/pubsub/PubsubClient.java | 28 +++++++++++++++++++ .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 9 ++++++ .../io/gcp/pubsub/PubsubUnboundedSink.java | 13 +++++++++ .../io/gcp/pubsub/PubsubUnboundedSource.java | 14 ++++++++++ .../sdk/io/gcp/pubsub/PubsubClientTest.java | 2 ++ .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java | 3 ++ 6 files changed, 69 insertions(+) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java index 79a9bb7f07d6..f66ee6e1d842 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java @@ -39,12 +39,15 @@ 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.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** An (abstract) helper class for talking to Pubsub via an underlying transport. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public abstract class PubsubClient implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(PubsubClient.class); private static final Map> schemaTypeToConversionFnMap = ImmutableMap.of( @@ -257,6 +260,10 @@ public String getFullPath() { return String.format("/subscriptions/%s/%s", projectId, subscriptionName); } + public String getDataCatalogName() { + return String.format("pubsub:subscription:%s.%s", projectId, subscriptionName); + } + @Override public boolean equals(@Nullable Object o) { if (this == o) { @@ -293,6 +300,7 @@ public static SubscriptionPath subscriptionPathFromName( /** Path representing a Pubsub topic. */ public static class TopicPath implements Serializable { + // Format: "projects//topics/" private final String path; TopicPath(String path) { @@ -310,6 +318,26 @@ public String getName() { return splits.get(3); } + /** + * Returns the data catalog name. Format "pubsub:topic:`project`.`topic`" This method is + * fail-safe. If topic path is malformed, it returns an empty string. + */ + public String getDataCatalogName() { + List splits = Splitter.on('/').splitToList(path); + if (splits.size() == 4) { + // well-formed path + return String.format("pubsub:topic:%s.%s", splits.get(1), splits.get(3)); + } else { + // Mal-formed path. It is either a test fixture or user error and will fail on publish. + // We do not throw exception instead return empty string here. + LOG.warn( + "Cannot get data catalog name for malformed topic path {}. Expected format: " + + "projects//topics/", + path); + return ""; + } + } + public String getFullPath() { List splits = Splitter.on('/').splitToList(path); checkState(splits.size() == 4, "Malformed topic path %s", path); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index 6233cf669080..0fd4e9207d81 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -49,6 +49,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; @@ -512,6 +513,10 @@ public String asPath() { } } + public String dataCatalogName() { + return String.format("pubsub:topic:%s.%s", project, topic); + } + @Override public String toString() { return asPath(); @@ -1617,6 +1622,10 @@ public void finishBundle() throws IOException { for (Map.Entry entry : output.entrySet()) { publish(entry.getKey(), entry.getValue().messages); } + // Report lineage for all topics seen + for (PubsubTopic topic : output.keySet()) { + Lineage.getSinks().add(topic.dataCatalogName()); + } output = null; pubsubClient.close(); pubsubClient = null; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index aa8e3a411486..defea87e835a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.PubsubClientFactory; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.metrics.SinkMetrics; import org.apache.beam.sdk.options.ValueProvider; @@ -69,6 +70,7 @@ import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.hash.Hashing; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -231,6 +233,9 @@ private static class WriterFn extends DoFn, Void> { /** Client on which to talk to Pubsub. Null until created by {@link #startBundle}. */ private transient @Nullable PubsubClient pubsubClient; + /** Last TopicPath that reported Lineage. */ + private transient @Nullable TopicPath reportedLineage; + private final Counter batchCounter = Metrics.counter(WriterFn.class, "batches"); private final Counter elementCounter = SinkMetrics.elementsWritten(); private final Counter byteCounter = SinkMetrics.bytesWritten(); @@ -290,6 +295,14 @@ private void publishBatch(List messages, int bytes) throws IOEx batchCounter.inc(); elementCounter.inc(messages.size()); byteCounter.inc(bytes); + // Report Lineage multiple once for same topic + if (!topicPath.equals(reportedLineage)) { + String name = topicPath.getDataCatalogName(); + if (!Strings.isNullOrEmpty(name)) { + Lineage.getSinks().add(topicPath.getDataCatalogName()); + } + reportedLineage = topicPath; + } } @StartBundle diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index b9a554d54ade..b131b521c067 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -56,6 +56,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessages.DeserializeBytesIntoPubsubMessagePayloadOnly; import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Lineage; import org.apache.beam.sdk.metrics.SourceMetrics; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -1041,6 +1042,19 @@ public List split(int desiredNumSplits, PipelineOptions options) splitSource = new PubsubSource( outer, StaticValueProvider.of(outer.createRandomSubscription(options))); + TopicPath topic = outer.getTopic(); + if (topic != null) { + // is initial split on Read.fromTopic, report Lineage based on topic + Lineage.getSources().add(topic.getDataCatalogName()); + } + } else { + if (subscriptionPath.equals(outer.getSubscriptionProvider())) { + SubscriptionPath sub = subscriptionPath.get(); + if (sub != null) { + // is a split on Read.fromSubscription + Lineage.getSources().add(sub.getDataCatalogName()); + } + } } for (int i = 0; i < desiredNumSplits * SCALE_OUT; i++) { // Since the source is immutable and Pubsub automatically shards we simply diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java index 895ed35bfb12..fb007d1171db 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClientTest.java @@ -171,6 +171,7 @@ public void subscriptionPathFromNameWellFormed() { SubscriptionPath path = PubsubClient.subscriptionPathFromName("test", "something"); assertEquals("projects/test/subscriptions/something", path.getPath()); assertEquals("/subscriptions/test/something", path.getFullPath()); + assertEquals("pubsub:subscription:test.something", path.getDataCatalogName()); } @Test @@ -178,6 +179,7 @@ public void topicPathFromNameWellFormed() { TopicPath path = PubsubClient.topicPathFromName("test", "something"); assertEquals("projects/test/topics/something", path.getPath()); assertEquals("/topics/test/something", path.getFullPath()); + assertEquals("pubsub:topic:test.something", path.getDataCatalogName()); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index 3027db6aee9d..74a98f0b8b43 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -237,6 +237,9 @@ public void testValueProviderTopic() { assertThat(pubsubRead.getTopicProvider(), not(nullValue())); assertThat(pubsubRead.getTopicProvider().isAccessible(), is(true)); assertThat(pubsubRead.getTopicProvider().get().asPath(), equalTo(provider.get())); + assertThat( + pubsubRead.getTopicProvider().get().dataCatalogName(), + equalTo("pubsub:topic:project.topic")); } @Test From e3e4454457762c85ca7c8068f0e9f2e20966dccc Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 6 Aug 2024 12:54:31 -0700 Subject: [PATCH 13/21] [#32085][prism] Fix session windowing. (#32086) --- .../runners/prism/internal/handlerunner.go | 24 ++++++++++++------- .../runners/portability/prism_runner_test.py | 22 +++++++++++++++++ 2 files changed, 37 insertions(+), 9 deletions(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go index a1eeeba02c4b..eecebde3d693 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlerunner.go @@ -244,7 +244,7 @@ func (h *runner) ExecuteTransform(stageID, tid string, t *pipepb.PTransform, com kc := coders[kcID] ec := coders[ecID] - data = append(data, gbkBytes(ws, wc, kc, ec, inputData, coders, watermark)) + data = append(data, gbkBytes(ws, wc, kc, ec, inputData, coders)) if len(data[0]) == 0 { panic("no data for GBK") } @@ -290,7 +290,7 @@ func windowingStrategy(comps *pipepb.Components, tid string) *pipepb.WindowingSt } // gbkBytes re-encodes gbk inputs in a gbk result. -func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregate [][]byte, coders map[string]*pipepb.Coder, watermark mtime.Time) []byte { +func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregate [][]byte, coders map[string]*pipepb.Coder) []byte { // Pick how the timestamp of the aggregated output is computed. var outputTime func(typex.Window, mtime.Time, mtime.Time) mtime.Time switch ws.GetOutputTime() { @@ -333,9 +333,8 @@ func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregat kd := pullDecoder(kc, coders) vd := pullDecoder(vc, coders) - // Right, need to get the key coder, and the element coder. - // Cus I'll need to pull out anything the runner knows how to deal with. - // And repeat. + // Aggregate by windows and keys, using the window coder and KV coders. + // We need to extract and split the key bytes from the element bytes. for _, data := range toAggregate { // Parse out each element's data, and repeat. buf := bytes.NewBuffer(data) @@ -388,34 +387,41 @@ func gbkBytes(ws *pipepb.WindowingStrategy, wc, kc, vc *pipepb.Coder, toAggregat } // Use a decreasing sort (latest to earliest) so we can correct // the output timestamp to the new end of window immeadiately. - // TODO need to correct this if output time is different. sort.Slice(ordered, func(i, j int) bool { return ordered[i].MaxTimestamp() > ordered[j].MaxTimestamp() }) cur := ordered[0] sessionData := windows[cur] + delete(windows, cur) for _, iw := range ordered[1:] { - // If they overlap, then we merge the data. + // Check if the gap between windows is less than the gapSize. + // If not, this window is done, and we start a next window. if iw.End+gapSize < cur.Start { - // Start a new session. + // Store current data with the current window. windows[cur] = sessionData + // Use the incoming window instead, and clear it from the map. cur = iw sessionData = windows[iw] + delete(windows, cur) + // There's nothing to merge, since we've just started with this windowed data. continue } - // Extend the session + // Extend the session with the incoming window, and merge the the incoming window's data. cur.Start = iw.Start toMerge := windows[iw] delete(windows, iw) for k, kt := range toMerge { skt := sessionData[k] + // Ensure the output time matches the given function. + skt.time = outputTime(cur, kt.time, skt.time) skt.key = kt.key skt.w = cur skt.values = append(skt.values, kt.values...) sessionData[k] = skt } } + windows[cur] = sessionData } // Everything's aggregated! // Time to turn things into a windowed KV> diff --git a/sdks/python/apache_beam/runners/portability/prism_runner_test.py b/sdks/python/apache_beam/runners/portability/prism_runner_test.py index f1ccf66a2289..324fe5a17b54 100644 --- a/sdks/python/apache_beam/runners/portability/prism_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/prism_runner_test.py @@ -40,7 +40,9 @@ from apache_beam.runners.portability import portable_runner_test from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms import window from apache_beam.transforms.sql import SqlTransform +from apache_beam.utils import timestamp # Run as # @@ -178,6 +180,26 @@ def create_options(self): return options + # Slightly more robust session window test: + # Validates that an inner grouping doesn't duplicate data either. + # Copied also because the timestamp in fn_runner_test.py isn't being + # inferred correctly as seconds for some reason, but as micros. + # The belabored specification is validating the timestamp type works at least. + # See https://github.com/apache/beam/issues/32085 + def test_windowing(self): + with self.create_pipeline() as p: + res = ( + p + | beam.Create([1, 2, 100, 101, 102, 123]) + | beam.Map( + lambda t: window.TimestampedValue( + ('k', t), timestamp.Timestamp.of(t).micros)) + | beam.WindowInto(beam.transforms.window.Sessions(10)) + | beam.GroupByKey() + | beam.Map(lambda k_vs1: (k_vs1[0], sorted(k_vs1[1])))) + assert_that( + res, equal_to([('k', [1, 2]), ('k', [100, 101, 102]), ('k', [123])])) + # Can't read host files from within docker, read a "local" file there. def test_read(self): print('name:', __name__) From 0a42afa9f5c02e9d529e1c1f1b197472a44cc174 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Tue, 6 Aug 2024 16:10:08 -0700 Subject: [PATCH 14/21] [prism] Use non-deprecated docker types in environment. (#32092) * [prism] Use non-deprecated docker types in environment. * Include Go mod changes. * Update testcontainers-go * revert toolchain change * go mod tidy requirement, update minimum Go version. * Note Minimum Go version update. --------- Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- CHANGES.md | 1 + sdks/go.mod | 20 +++--- sdks/go.sum | 70 ++++++++++++++----- .../runners/prism/internal/environments.go | 8 +-- 4 files changed, 69 insertions(+), 30 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 7f12b5334260..129fa01f94a8 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,6 +68,7 @@ ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Go SDK Minimum Go Version updated to 1.21 ([#32092](https://github.com/apache/beam/pull/32092)). ## Breaking Changes diff --git a/sdks/go.mod b/sdks/go.mod index 1716a6e2d22d..7eb29b3cc77c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -20,7 +20,7 @@ // directory. module github.com/apache/beam/sdks/v2 -go 1.20 +go 1.21 require ( cloud.google.com/go/bigquery v1.61.0 @@ -49,7 +49,7 @@ require ( github.com/nats-io/nats.go v1.35.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.1 - github.com/testcontainers/testcontainers-go v0.26.0 + github.com/testcontainers/testcontainers-go v0.32.0 github.com/tetratelabs/wazero v1.7.3 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c @@ -79,21 +79,23 @@ require ( dario.cat/mergo v1.0.0 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 // indirect - github.com/Microsoft/hcsshim v0.11.4 // indirect + github.com/Microsoft/hcsshim v0.11.5 // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect + github.com/containerd/errdefs v0.1.0 // indirect github.com/containerd/log v0.1.0 // indirect - github.com/distribution/reference v0.5.0 // indirect + github.com/distribution/reference v0.6.0 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/minio/highwayhash v1.0.2 // indirect + github.com/moby/docker-image-spec v1.3.1 // indirect github.com/moby/sys/user v0.1.0 // indirect github.com/nats-io/jwt/v2 v2.5.7 // indirect github.com/nats-io/nkeys v0.4.7 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect - github.com/shirou/gopsutil/v3 v3.23.9 // indirect + github.com/shirou/gopsutil/v3 v3.23.12 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect @@ -114,7 +116,7 @@ require ( cloud.google.com/go/iam v1.1.8 // indirect cloud.google.com/go/longrunning v0.5.7 // indirect github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect - github.com/Microsoft/go-winio v0.6.1 // indirect + github.com/Microsoft/go-winio v0.6.2 // indirect github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 // indirect github.com/apache/thrift v0.17.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect @@ -135,9 +137,9 @@ require ( github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect - github.com/containerd/containerd v1.7.11 // indirect + github.com/containerd/containerd v1.7.18 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect - github.com/docker/docker v25.0.6+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v27.1.1+incompatible // 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.12.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.0.4 // indirect @@ -165,7 +167,7 @@ require ( github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe // indirect github.com/morikuni/aec v1.0.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect - github.com/opencontainers/image-spec v1.1.0-rc5 // indirect + github.com/opencontainers/image-spec v1.1.0 // indirect github.com/pierrec/lz4/v4 v4.1.18 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pkg/xattr v0.4.9 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index a0b4738decc3..ce10d84dd044 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -209,6 +209,7 @@ cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnR cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= cloud.google.com/go/datacatalog v1.20.1 h1:czcba5mxwRM5V//jSadyig0y+8aOHmN7gUl9GbHu59E= +cloud.google.com/go/datacatalog v1.20.1/go.mod h1:Jzc2CoHudhuZhpv78UBAjMEg3w7I9jHA11SbRshWUjk= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= @@ -346,6 +347,7 @@ cloud.google.com/go/kms v1.9.0/go.mod h1:qb1tPTgfF9RQP8e1wq4cLFErVuTJv7UsSC915J8 cloud.google.com/go/kms v1.10.0/go.mod h1:ng3KTUtQQU9bPX3+QGLsflZIHlkbn8amFAMY63m8d24= cloud.google.com/go/kms v1.10.1/go.mod h1:rIWk/TryCkR59GMC3YtHtXeLzd634lBbKenvyySAyYI= cloud.google.com/go/kms v1.18.0 h1:pqNdaVmZJFP+i8OVLocjfpdTWETTYa20FWOegSCdrRo= +cloud.google.com/go/kms v1.18.0/go.mod h1:DyRBeWD/pYBMeyiaXFa/DGNyxMDL3TslIKb8o/JkLkw= cloud.google.com/go/language v1.4.0/go.mod h1:F9dRpNFQmJbkaop6g0JhSBXCNlO90e1KWx5iDdxbWic= cloud.google.com/go/language v1.6.0/go.mod h1:6dJ8t3B+lUYfStgls25GusK04NLh3eDLQnWM3mdEbhI= cloud.google.com/go/language v1.7.0/go.mod h1:DJ6dYN/W+SQOjF8e1hLQXMF21AkH2w9wiPzPCJa2MIE= @@ -628,6 +630,7 @@ filippo.io/edwards25519 v1.1.0/go.mod h1:BxyFTGdWcka3PhytdK4V28tE5sGfRvvvRV7EaN4 gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zumjgTw83q2ge/PI+yyw8= git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 h1:bvDV9vkmnHYOMsOr4WLk+Vo07yKIzd94sVoIqshQ4bU= +github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h1:8o94RPi1/7XTJvwPpRSzSUedZrtlirdB3r9Z20bi2f8= github.com/Azure/azure-pipeline-go v0.2.3/go.mod h1:x841ezTBIMG6O3lAcl8ATHnsOPVl2bqk7S3ta6S6u4k= github.com/Azure/azure-storage-blob-go v0.14.0/go.mod h1:SMqIBi+SuiQH32bvyjngEewEeXoPfKMgWlBDaYf6fck= github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOElx5B5HZ4hJQsoJ/PvUvKRhJHDQXO8P8= @@ -643,10 +646,10 @@ github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0 h1:oVLqHXhnYtUwM89y9T1fXGaK9wTkXHgNp8/ZNMQzUxE= github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.0/go.mod h1:dppbR7CwXD4pgtV9t3wD1812RaLDcBjtblcDF5f1vI0= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= -github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= -github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/Microsoft/hcsshim v0.11.4 h1:68vKo2VN8DE9AdN4tnkWnmdhqdbpUFM8OF3Airm7fz8= -github.com/Microsoft/hcsshim v0.11.4/go.mod h1:smjE4dvqPX9Zldna+t5FG3rnoHhaB7QYxPRqGcpAD9w= +github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= +github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU= +github.com/Microsoft/hcsshim v0.11.5 h1:haEcLNpj9Ka1gd3B3tAEs9CpE0c+1IhoL59w/exYU38= +github.com/Microsoft/hcsshim v0.11.5/go.mod h1:MV8xMfmECjl5HdO7U/3/hFVnkmSBjAjmA09d4bExKcU= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY= github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk= @@ -754,8 +757,10 @@ github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 h1:DBmgJDC9dTfkVyGgipamEh2BpGYxScCH1TOF1LL1cXc= github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50/go.mod h1:5e1+Vvlzido69INQaVO6d87Qn543Xr6nooe9Kz7oBFM= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= -github.com/containerd/containerd v1.7.11 h1:lfGKw3eU35sjV0aG2eYZTiwFEY1pCzxdzicHP3SZILw= -github.com/containerd/containerd v1.7.11/go.mod h1:5UluHxHTX2rdvYuZ5OJTC5m/KJNs0Zs9wVoJm9zf5ZE= +github.com/containerd/containerd v1.7.18 h1:jqjZTQNfXGoEaZdW1WwPU0RqSn1Bm2Ay/KJPUuO8nao= +github.com/containerd/containerd v1.7.18/go.mod h1:IYEk9/IO6wAPUz2bCMVUbsfXjzw5UNP5fLz4PsUygQ4= +github.com/containerd/errdefs v0.1.0 h1:m0wCRBiu1WJT/Fr+iOoQHMQS/eP5myQ8lCv4Dz5ZURM= +github.com/containerd/errdefs v0.1.0/go.mod h1:YgWiiHtLmSeBrvpw+UfPijzbLaB77mEG1WwJTDETIV0= github.com/containerd/log v0.1.0 h1:TCJt7ioM2cr/tfR8GPbGf9/VRAX8D2B4PjzCpfX540I= github.com/containerd/log v0.1.0/go.mod h1:VRRf09a7mHDIRezVKTRCrOq78v577GXq3bSa3EhrzVo= github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= @@ -763,13 +768,14 @@ github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHf github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= +github.com/creack/pty v1.1.18/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= -github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/docker/docker v25.0.6+incompatible h1:5cPwbwriIcsua2REJe8HqQV+6WlWc1byg2QSXzBxBGg= -github.com/docker/docker v25.0.6+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/distribution/reference v0.6.0 h1:0IXCQ5g4/QMHHkarYzh5l+u8T3t73zM5QvfrDyIgxBk= +github.com/distribution/reference v0.6.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= +github.com/docker/docker v27.1.1+incompatible h1:hO/M4MtV36kzKldqnA37IWhebRA+LnqqcqDja6kVaKY= +github.com/docker/docker v27.1.1+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= @@ -814,6 +820,7 @@ github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9 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= github.com/go-gorp/gorp v2.2.0+incompatible h1:xAUh4QgEeqPPhK3vxZN+bzrim1z5Av6q837gtjUlshc= +github.com/go-gorp/gorp v2.2.0+incompatible/go.mod h1:7IfkAQnO7jfT/9IQ3R9wL1dFhukN6aQxzKTHnkxzA/E= github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U= github.com/go-latex/latex v0.0.0-20210823091927-c0d11ff05a81/go.mod h1:SX0U8uGpxhq9o2S/CELCSUxEWWAuoCUcVCQWv7G2OCk= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= @@ -881,6 +888,7 @@ github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= +github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/flatbuffers v2.0.8+incompatible/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/flatbuffers v23.5.26+incompatible h1:M9dgRyhJemaM4Sw8+66GHBu8ioaQmyPLg1b8VwK5WJg= @@ -909,6 +917,7 @@ github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIG github.com/google/martian/v3 v3.2.1/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/martian/v3 v3.3.2/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= github.com/google/martian/v3 v3.3.3 h1:DIhPTQrbPkgs2yJYdXU/eNACCG5DVQjySNRNlflZ9Fc= +github.com/google/martian/v3 v3.3.3/go.mod h1:iEPrYcgCF7jA9OtScMFQyAlZZ4YXTKEtJ1E6RWzmBA0= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -967,6 +976,7 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFb github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU0GSB0f8Nhgmxx0VIRUvaC0w= github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0 h1:YBftPWNWd4WwGqtY2yeZL2ef8rHAxPBD8KFhJpmcqms= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg= 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= @@ -976,11 +986,16 @@ github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1: github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsIM= +github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg= github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a h1:bbPeKD0xmW/Y25WS6cokEszi5g+S0QxI/d45PkRi7Nk= +github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM= github.com/jackc/pgx/v5 v5.3.1 h1:Fcr8QJ1ZeLi5zsPZqQeUZhNhxfkkKBOgJuYkJHoBOtU= +github.com/jackc/pgx/v5 v5.3.1/go.mod h1:t3JDKnCBlYIc0ewLF0Q7B8MXmoIaBOZj/ic7iHozM/8= github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jinzhu/inflection v1.0.0 h1:K317FqzuhWc8YvSVlFMCCUb36O/S9MCKRDI7QkRKD/E= +github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= github.com/jinzhu/now v1.1.5 h1:/o9tlHleP7gOFmsnYNz3RGnqzefHA47wQpKrrdTIwXQ= +github.com/jinzhu/now v1.1.5/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= @@ -990,6 +1005,7 @@ github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfC github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 h1:eQGUsj2LcsLzfrHY1noKDSU7h+c9/rw9pQPwbQ9g1jQ= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6/go.mod h1:LIAXxPvcUXwOcTIj9LSNSUpE9/eMHalTWxsP/kmWxQI= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= @@ -1012,6 +1028,7 @@ github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORN github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -1036,8 +1053,13 @@ github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8Ie github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= +github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= +github.com/minio/minio-go/v7 v7.0.66/go.mod h1:DHAgmyQEGdW3Cif0UooKOyrT3Vxs82zNdV6tkKhRtbs= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= +github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= +github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0= +github.com/moby/docker-image-spec v1.3.1/go.mod h1:eKmb5VW8vQEh/BAr2yvVNvuiJuY6UIocYsFu/DxxRpo= github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= github.com/moby/patternmatcher v0.6.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5lXtc= @@ -1047,7 +1069,9 @@ github.com/moby/sys/user v0.1.0/go.mod h1:fKJhFOnsCN6xZ5gSfbM6zaHGgDJMrqt9/reuj4 github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe h1:iruDEfMl2E6fbMZ9s0scYfZQ84/6SPL6zC8ACM2oIL0= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= @@ -1065,8 +1089,8 @@ github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OS github.com/ncw/swift v1.0.52/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= -github.com/opencontainers/image-spec v1.1.0-rc5 h1:Ygwkfw9bpDvs+c9E34SdgGOj41dX/cbdlwvlWt0pnFI= -github.com/opencontainers/image-spec v1.1.0-rc5/go.mod h1:X4pATf0uXsnn3g5aiGIsVnJBR4mxhKzfwmvK/B2NTm8= +github.com/opencontainers/image-spec v1.1.0 h1:8SG7/vwALn54lVB/0yZ/MMwhFrPYtpEHQb2IpWsCzug= +github.com/opencontainers/image-spec v1.1.0/go.mod h1:W4s4sFTMaBeK1BQLXbG4AdM2szdn85PY75RI83NrTrM= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY= github.com/phpdave11/gofpdi v1.0.12/go.mod h1:vBmVV0Do6hSBHC8uKUQ71JGW+ZGQq74llk/7bXwjDoI= @@ -1099,6 +1123,7 @@ github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTE github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc= +github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfFZQK844Gfx8o5WFuvpxWRwnSoipWe/p622j1v06w= github.com/ruudk/golang-pdf417 v0.0.0-20201230142125-a7e3863a1245/go.mod h1:pQAZKsJ8yyVxGRWYNEm9oFB8ieLgKFnamEyDmSA0BRk= @@ -1106,8 +1131,8 @@ github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 h1:GHRpF1pTW19a github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5PCi+MFsC7HjREoAz1BU+Mq60+05gifQSsHSDG/8= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63 h1:J6qvD6rbmOil46orKqJaRPG+zTpoGlBTUdyv8ki63L0= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63/go.mod h1:n+VKSARF5y/tS9XFSP7vWDfS+GUC5vs/YT7M5XDTUEM= -github.com/shirou/gopsutil/v3 v3.23.9 h1:ZI5bWVeu2ep4/DIxB4U9okeYJ7zp/QLTO4auRb/ty/E= -github.com/shirou/gopsutil/v3 v3.23.9/go.mod h1:x/NWSb71eMcjFIO0vhyGW5nZ7oSIgVjrCnADckb85GA= +github.com/shirou/gopsutil/v3 v3.23.12 h1:z90NtUkp3bMtmICZKpC4+WaknU1eXtp5vtbQ11DgpE4= +github.com/shirou/gopsutil/v3 v3.23.12/go.mod h1:1FrWgea594Jp7qmjHUUPlJDTPgcsb9mGnXDxavtikzM= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= @@ -1141,8 +1166,9 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o github.com/stretchr/testify v1.8.3/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/testcontainers/testcontainers-go v0.26.0 h1:uqcYdoOHBy1ca7gKODfBd9uTHVK3a7UL848z09MVZ0c= -github.com/testcontainers/testcontainers-go v0.26.0/go.mod h1:ICriE9bLX5CLxL9OFQ2N+2N+f+803LNJ1utJb1+Inx0= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/testcontainers/testcontainers-go v0.32.0 h1:ug1aK08L3gCHdhknlTTwWjPHPS+/alvLJU/DRxTD/ME= +github.com/testcontainers/testcontainers-go v0.32.0/go.mod h1:CRHrzHLQhlXUsa5gXjTOfqIEJcrK5+xMDmBr/WMI88E= github.com/tetratelabs/wazero v1.7.3 h1:PBH5KVahrt3S2AHgEjKu4u+LlDbbk+nsGE3KLucy6Rw= github.com/tetratelabs/wazero v1.7.3/go.mod h1:ytl6Zuh20R/eROuyDaGPkp82O9C/DJfXAwJfQ3X6/7Y= github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= @@ -1198,7 +1224,9 @@ go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0/go.mod h1: go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo= go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0 h1:Mne5On7VWdx7omSrSSZvM4Kw7cS7NQkOOmLcgscI51U= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.19.0/go.mod h1:IPtUMKL4O3tH5y+iXVyAXqpAwMuzC1IrxVS81rummfE= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0 h1:IeMeyr1aBvBiPVYihXIaeIZba6b8E1bYp7lbdxK8CQg= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.19.0/go.mod h1:oVdCUtjq9MK9BlS7TtucsQwUcXcymNiEDjgDD2jMtZU= go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGXlc88kI= go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco= go.opentelemetry.io/otel/sdk v1.24.0 h1:YMPPDNymmQN3ZgczicBY3B6sf9n62Dlj9pWD3ucgoDw= @@ -1209,6 +1237,7 @@ go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqe go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= +go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= 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= @@ -1482,7 +1511,7 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -1493,6 +1522,8 @@ golang.org/x/term v0.4.0/go.mod h1:9P2UbLfCdcvo3p/nzKvsmas4TnlujnuoV9hGgYzW1lQ= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= +golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= +golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1600,6 +1631,7 @@ gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/gonum v0.9.3/go.mod h1:TZumC3NeyVQskjXqmyWt4S3bINhy7B4eYwW69EbyX+0= gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= gonum.org/v1/gonum v0.12.0 h1:xKuo6hzt+gMav00meVPUlXwSdoEJP46BR+wdxQEFK2o= +gonum.org/v1/gonum v0.12.0/go.mod h1:73TDxJfAAHeA8Mk9mf8NlIppyhQNo5GLTcYeqgo2lvY= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= gonum.org/v1/plot v0.9.0/go.mod h1:3Pcqqmp6RHvJI72kgb8fThyUnav364FOsdDo2aGW5lY= @@ -1877,6 +1909,7 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= @@ -1892,8 +1925,11 @@ gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gorm.io/driver/postgres v1.5.2 h1:ytTDxxEv+MplXOfFe3Lzm7SjG09fcdb3Z/c056DTBx0= +gorm.io/driver/postgres v1.5.2/go.mod h1:fmpX0m2I1PKuR7mKZiEluwrP3hbs+ps7JIGMUBpCgl8= gorm.io/gorm v1.25.2 h1:gs1o6Vsa+oVKG/a9ElL3XgyGfghFfkKA2SInQaCyMho= +gorm.io/gorm v1.25.2/go.mod h1:L4uxeKpfBml98NYqVqwAdmV1a2nBtAec/cf3fpucW/k= gotest.tools/v3 v3.5.1 h1:EENdUnS3pdur5nybKYIh2Vfgc8IUNBjxDPSjtiJcOzU= +gotest.tools/v3 v3.5.1/go.mod h1:isy3WKz7GK6uNw/sbHzfKBLvlvXwUyV06n6brMxxopU= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/sdks/go/pkg/beam/runners/prism/internal/environments.go b/sdks/go/pkg/beam/runners/prism/internal/environments.go index 3a429920fb28..add7f769a702 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/environments.go +++ b/sdks/go/pkg/beam/runners/prism/internal/environments.go @@ -32,8 +32,8 @@ import ( "google.golang.org/grpc/credentials/insecure" "google.golang.org/protobuf/proto" - dtyp "github.com/docker/docker/api/types" "github.com/docker/docker/api/types/container" + "github.com/docker/docker/api/types/image" "github.com/docker/docker/api/types/mount" dcli "github.com/docker/docker/client" "github.com/docker/docker/pkg/stdcopy" @@ -132,7 +132,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock } if _, _, err := cli.ImageInspectWithRaw(ctx, dp.GetContainerImage()); err != nil { // We don't have a local image, so we should pull it. - if rc, err := cli.ImagePull(ctx, dp.GetContainerImage(), dtyp.ImagePullOptions{}); err == nil { + if rc, err := cli.ImagePull(ctx, dp.GetContainerImage(), image.PullOptions{}); err == nil { // Copy the output, but discard it so we can wait until the image pull is finished. io.Copy(io.Discard, rc) rc.Close() @@ -164,7 +164,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock containerID := ccr.ID logger = logger.With("container", containerID) - if err := cli.ContainerStart(ctx, containerID, dtyp.ContainerStartOptions{}); err != nil { + if err := cli.ContainerStart(ctx, containerID, container.StartOptions{}); err != nil { cli.Close() return fmt.Errorf("unable to start container image %v with docker for env %v, err: %w", dp.GetContainerImage(), wk.Env, err) } @@ -189,7 +189,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock case resp := <-statusCh: logger.Info("docker container has self terminated", "status_code", resp.StatusCode) - rc, err := cli.ContainerLogs(ctx, containerID, dtyp.ContainerLogsOptions{Details: true, ShowStdout: true, ShowStderr: true}) + rc, err := cli.ContainerLogs(ctx, containerID, container.LogsOptions{Details: true, ShowStdout: true, ShowStderr: true}) if err != nil { logger.Error("docker container logs error", "error", err) } From 9b564ef925b83ca040c46d54314c600f5e65940c Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Tue, 6 Aug 2024 17:33:29 -0700 Subject: [PATCH 15/21] Exclude a not yet implemented pandas op from dataframe tests. (#32066) --- sdks/python/apache_beam/dataframe/pandas_doctests_test.py | 1 + sdks/python/setup.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py index a4bd0d0a8127..c7ea908a9336 100644 --- a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py +++ b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py @@ -557,6 +557,7 @@ def test_series_tests(self): 'pandas.core.series.Series': ['ser.iloc[0] = 999'], }, not_implemented_ok={ + 'pandas.core.series.Series.case_when': ['*'], 'pandas.core.series.Series.transform': [ # str arg not supported. Tested with np.sum in # frames_test.py::DeferredFrameTest::test_groupby_transform_sum diff --git a/sdks/python/setup.py b/sdks/python/setup.py index c9b2d087d04c..756c952b0101 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -155,7 +155,7 @@ def cythonize(*args, **kwargs): # Exclude 1.5.0 and 1.5.1 because of # https://github.com/pandas-dev/pandas/issues/45725 dataframe_dependency = [ - 'pandas>=1.4.3,!=1.5.0,!=1.5.1,<2.2;python_version>="3.8"', + 'pandas>=1.4.3,!=1.5.0,!=1.5.1,<2.3;python_version>="3.8"', ] From eeddc6924c3230f6b502af67914918730a27efc9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 17:36:59 -0700 Subject: [PATCH 16/21] Bump google.golang.org/grpc from 1.64.0 to 1.65.0 in /sdks (#31824) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.64.0 to 1.65.0. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.64.0...v1.65.0) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 7 ++++--- sdks/go.sum | 13 ++++++++----- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 7eb29b3cc77c..9cdad36010e4 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -61,7 +61,7 @@ require ( golang.org/x/text v0.16.0 google.golang.org/api v0.187.0 google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d - google.golang.org/grpc v1.64.1 + google.golang.org/grpc v1.65.0 google.golang.org/protobuf v1.34.2 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 @@ -74,6 +74,7 @@ require ( ) require ( + cel.dev/expr v0.15.0 // indirect cloud.google.com/go/auth v0.6.1 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.2 // indirect dario.cat/mergo v1.0.0 // indirect @@ -135,8 +136,8 @@ require ( github.com/aws/aws-sdk-go-v2/service/sts v1.28.12 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect - github.com/cespare/xxhash/v2 v2.2.0 // indirect - github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b // indirect github.com/containerd/containerd v1.7.18 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/docker/docker v27.1.1+incompatible // but required to resolve issue docker has with go1.20 diff --git a/sdks/go.sum b/sdks/go.sum index ce10d84dd044..935009cf1d83 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1,3 +1,5 @@ +cel.dev/expr v0.15.0 h1:O1jzfJCQBfL5BFoYktaxwIhuttaQPsVWerH9/EEKx0w= +cel.dev/expr v0.15.0/go.mod h1:TRSuuV7DlVCE/uwv5QbAiW/v8l5O8C4eEPHeu7gf7Sg= cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= @@ -735,8 +737,9 @@ github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMr github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= 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/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -754,8 +757,8 @@ github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50 h1:DBmgJDC9dTfkVyGgipamEh2BpGYxScCH1TOF1LL1cXc= -github.com/cncf/xds/go v0.0.0-20240318125728-8a4994d93e50/go.mod h1:5e1+Vvlzido69INQaVO6d87Qn543Xr6nooe9Kz7oBFM= +github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b h1:ga8SEFjZ60pxLcmhnThWgvH2wg8376yUJmPhEH4H3kw= +github.com/cncf/xds/go v0.0.0-20240423153145-555b57ec207b/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/containerd/containerd v1.7.18 h1:jqjZTQNfXGoEaZdW1WwPU0RqSn1Bm2Ay/KJPUuO8nao= github.com/containerd/containerd v1.7.18/go.mod h1:IYEk9/IO6wAPUz2bCMVUbsfXjzw5UNP5fLz4PsUygQ4= @@ -1881,8 +1884,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.64.1 h1:LKtvyfbX3UGVPFcGqJ9ItpVWW6oN/2XqTxfAnwRRXiA= -google.golang.org/grpc v1.64.1/go.mod h1:hiQF4LFZelK2WKaP6W0L92zGHtiQdZxk8CrSdvyjeP0= +google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= +google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= From 99672af7fe12f72b562289d0c9449e4711b973c2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 17:38:40 -0700 Subject: [PATCH 17/21] Bump torch from 1.13.1 to 2.2.0 in /sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train (#31983) Bumps [torch](https://github.com/pytorch/pytorch) from 1.13.1 to 2.2.0. - [Release notes](https://github.com/pytorch/pytorch/releases) - [Changelog](https://github.com/pytorch/pytorch/blob/main/RELEASE.md) - [Commits](https://github.com/pytorch/pytorch/compare/v1.13.1...v2.2.0) --- updated-dependencies: - dependency-name: torch dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../ml-orchestration/kfp/components/train/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt index 2e65f0fba246..ba1103dd1ef9 100644 --- a/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt +++ b/sdks/python/apache_beam/examples/ml-orchestration/kfp/components/train/requirements.txt @@ -13,6 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -torch==1.13.1 +torch==2.2.0 numpy==1.22.4 Pillow==10.2.0 \ No newline at end of file From ebba3bb026b63f05d358a2cb5608cf61107ee504 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 22:23:26 -0700 Subject: [PATCH 18/21] Bump go.mongodb.org/mongo-driver from 1.13.1 to 1.16.0 in /sdks (#32097) Bumps [go.mongodb.org/mongo-driver](https://github.com/mongodb/mongo-go-driver) from 1.13.1 to 1.16.0. - [Release notes](https://github.com/mongodb/mongo-go-driver/releases) - [Commits](https://github.com/mongodb/mongo-go-driver/compare/v1.13.1...v1.16.0) --- updated-dependencies: - dependency-name: go.mongodb.org/mongo-driver dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 11 ++++------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 9cdad36010e4..2d638fc1998f 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,7 +53,7 @@ require ( github.com/tetratelabs/wazero v1.7.3 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.13.1 + go.mongodb.org/mongo-driver v1.16.0 golang.org/x/net v0.26.0 golang.org/x/oauth2 v0.21.0 golang.org/x/sync v0.7.0 @@ -165,7 +165,7 @@ require ( github.com/moby/patternmatcher v0.6.0 // indirect github.com/moby/sys/sequential v0.5.0 // indirect github.com/moby/term v0.5.0 // indirect - github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe // indirect + github.com/montanaflynn/stats v0.7.1 // indirect github.com/morikuni/aec v1.0.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.1.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 935009cf1d83..138c484884d5 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1019,7 +1019,6 @@ github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+o github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= -github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= github.com/klauspost/compress v1.17.8 h1:YcnTYrq7MikUT7k0Yb5eceMmALQPYBW/Xltxn0NAMnU= github.com/klauspost/compress v1.17.8/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= @@ -1075,8 +1074,8 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= -github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe h1:iruDEfMl2E6fbMZ9s0scYfZQ84/6SPL6zC8ACM2oIL0= -github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= +github.com/montanaflynn/stats v0.7.1 h1:etflOAAHORrCC44V+aR6Ftzort912ZU+YLiSTuV8eaE= +github.com/montanaflynn/stats v0.7.1/go.mod h1:etXPPgVO6n31NxCd9KQUMvCM+ve0ruNzt6R8Bnaayow= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/nats-io/jwt/v2 v2.5.7 h1:j5lH1fUXCnJnY8SsQeB/a/z9Azgu2bYIDvtPVNdxe2c= @@ -1209,8 +1208,8 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD go.einride.tech/aip v0.67.1 h1:d/4TW92OxXBngkSOwWS2CH5rez869KpKMaN44mdxkFI= go.einride.tech/aip v0.67.1/go.mod h1:ZGX4/zKw8dcgzdLsrvpOOGxfxI2QSk12SlP7d6c0/XI= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.mongodb.org/mongo-driver v1.13.1 h1:YIc7HTYsKndGK4RFzJ3covLz1byri52x0IoMB0Pt/vk= -go.mongodb.org/mongo-driver v1.13.1/go.mod h1:wcDf1JBCXy2mOW0bWHwO/IOYqdca1MPCwDtFu/Z9+eo= +go.mongodb.org/mongo-driver v1.16.0 h1:tpRsfBJMROVHKpdGyc1BBEzzjDUWjItxbVSZ8Ls4BQ4= +go.mongodb.org/mongo-driver v1.16.0/go.mod h1:oB6AhJQvFQL4LEHyXi6aJzQJtBiTQHiAd83l0GdFaiw= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -1253,7 +1252,6 @@ golang.org/x/crypto v0.0.0-20210421170649-83a5a9bb288b/go.mod h1:T9bdIzuCu7OtxOm golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20211108221036-ceb1ce70b4fa/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1357,7 +1355,6 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210813160813-60bc85c4be6d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220325170049-de3da57026de/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= From 44a9942719e754fbd1967e475466f404b9f47e22 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 7 Aug 2024 08:19:28 +0200 Subject: [PATCH 19/21] Add warning + doc callout when encountering ri pickling errors (#32063) --- sdks/python/apache_beam/ml/inference/base.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 401b57fdb807..29a568def07b 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -1586,6 +1586,15 @@ def _run_inference(self, batch, inference_args): except BaseException as e: if self._metrics_collector: self._metrics_collector.failed_batches_counter.inc() + if (e is pickle.PickleError and + self._model_handler.share_model_across_processes()): + raise TypeError( + 'Pickling error encountered while running inference. ' + 'This may be caused by trying to send unpickleable ' + 'data to a model which is shared across processes. ' + 'For more information, see ' + 'https://beam.apache.org/documentation/ml/large-language-modeling/#pickling-errors' # pylint: disable=line-too-long + ) from e raise e predictions = list(result_generator) From 0d81c5993049bc72116c2871c9b50bbe4cfc43d5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Aug 2024 10:08:13 -0400 Subject: [PATCH 20/21] Bump golang.org/x/text from 0.16.0 to 0.17.0 in /sdks (#32098) Bumps [golang.org/x/text](https://github.com/golang/text) from 0.16.0 to 0.17.0. - [Release notes](https://github.com/golang/text/releases) - [Commits](https://github.com/golang/text/compare/v0.16.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/text dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 2d638fc1998f..958a228c3546 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -56,9 +56,9 @@ require ( go.mongodb.org/mongo-driver v1.16.0 golang.org/x/net v0.26.0 golang.org/x/oauth2 v0.21.0 - golang.org/x/sync v0.7.0 + golang.org/x/sync v0.8.0 golang.org/x/sys v0.21.0 - golang.org/x/text v0.16.0 + golang.org/x/text v0.17.0 google.golang.org/api v0.187.0 google.golang.org/genproto v0.0.0-20240624140628-dc46fd24d27d google.golang.org/grpc v1.65.0 diff --git a/sdks/go.sum b/sdks/go.sum index 138c484884d5..ef3c436f22e0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1423,8 +1423,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1540,8 +1540,8 @@ golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= 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= From 81ad4fee378a586701dc0ff25bcc3c5cd7a9f3f6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Aug 2024 10:08:42 -0400 Subject: [PATCH 21/21] Bump github.com/aws/aws-sdk-go-v2/credentials in /sdks (#32096) Bumps [github.com/aws/aws-sdk-go-v2/credentials](https://github.com/aws/aws-sdk-go-v2) from 1.17.18 to 1.17.27. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/credentials/v1.17.18...credentials/v1.17.27) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/credentials dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 23 +++++++++++------------ sdks/go.sum | 45 ++++++++++++++++++++++----------------------- 2 files changed, 33 insertions(+), 35 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 958a228c3546..a5ad9f3b7f5c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.39.0 cloud.google.com/go/spanner v1.63.0 cloud.google.com/go/storage v1.43.0 - github.com/aws/aws-sdk-go-v2 v1.30.0 + github.com/aws/aws-sdk-go-v2 v1.30.3 github.com/aws/aws-sdk-go-v2/config v1.27.4 - github.com/aws/aws-sdk-go-v2/credentials v1.17.18 + github.com/aws/aws-sdk-go-v2/credentials v1.17.27 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 - github.com/aws/smithy-go v1.20.2 + github.com/aws/smithy-go v1.20.3 github.com/docker/go-connections v0.5.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.8.1 @@ -122,18 +122,18 @@ require ( github.com/apache/thrift v0.17.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.5 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.9 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.9 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.11 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.20.11 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.28.12 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.30.3 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect @@ -158,7 +158,6 @@ require ( github.com/gorilla/handlers v1.5.2 // indirect github.com/gorilla/mux v1.8.1 // indirect github.com/inconshreveable/mousetrap v1.1.0 // indirect - github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/compress v1.17.8 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect github.com/magiconair/properties v1.8.7 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index ef3c436f22e0..6afc175732ff 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -677,39 +677,39 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.30.0 h1:6qAwtzlfcTtcL8NHtbDQAqgM5s6NDipQTkPxyH/6kAA= -github.com/aws/aws-sdk-go-v2 v1.30.0/go.mod h1:ffIFB97e2yNsv4aTSGkqtHnppsIJzw7G7BReUZ3jCXM= +github.com/aws/aws-sdk-go-v2 v1.30.3 h1:jUeBtG0Ih+ZIFH0F4UkmL9w3cSpaMv9tYYDbzILP8dY= +github.com/aws/aws-sdk-go-v2 v1.30.3/go.mod h1:nIQjQVp5sfpQcTc9mPSr1B0PaWK5ByX9MOoDadSN4lc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= 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.27.4 h1:AhfWb5ZwimdsYTgP7Od8E9L1u4sKmDW2ZVeLcf2O42M= github.com/aws/aws-sdk-go-v2/config v1.27.4/go.mod h1:zq2FFXK3A416kiukwpsd+rD4ny6JC7QSkp4QdN1Mp2g= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.17.18 h1:D/ALDWqK4JdY3OFgA2thcPO1c9aYTT5STS/CvnkqY1c= -github.com/aws/aws-sdk-go-v2/credentials v1.17.18/go.mod h1:JuitCWq+F5QGUrmMPsk945rop6bB57jdscu+Glozdnc= +github.com/aws/aws-sdk-go-v2/credentials v1.17.27 h1:2raNba6gr2IfA0eqqiP2XiQ0UVOpGPgDSi0I9iAP+UI= +github.com/aws/aws-sdk-go-v2/credentials v1.17.27/go.mod h1:gniiwbGahQByxan6YjQUMcW4Aov6bLC3m+evgcoN4r4= 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.16.5 h1:dDgptDO9dxeFkXy+tEgVkzSClHZje/6JkPW5aZyEvrQ= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.5/go.mod h1:gjvE2KBUgUQhcv89jqxrIxH9GaKs1JbZzWejj/DaHGA= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11 h1:KreluoV8FZDEtI6Co2xuNk/UqI9iwMrOx/87PBNIKqw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.11/go.mod h1:SeSUYBLsMYFoRvHE0Tjvn7kbxaUhl75CJi1sbfhMxkU= 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.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.9 h1:cy8ahBJuhtM8GTTSyOkfy6WVPV1IE+SS5/wfXUYuulw= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.9/go.mod h1:CZBXGLaJnEZI6EVNcPd7a6B5IC5cA/GkRWtu9fp3S6Y= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.9 h1:A4SYk07ef04+vxZToz9LWvAXl9LW0NClpPpMsi31cz0= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.9/go.mod h1:5jJcHuwDagxN+ErjQ3PU3ocf6Ylc/p9x+BLO/+X4iXw= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 h1:SoNJ4RlFEQEbtDcCEt+QG56MY4fm4W8rYirAmq+/DdU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15/go.mod h1:U9ke74k1n2bf+RIgoX1SXFed1HLs51OgUSs+Ph0KJP8= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 h1:C6WHdGnTDIYETAm5iErQUiVNsclNx9qbJVPIt03B6bI= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15/go.mod h1:ZQLZqhcu+JhSrA9/NXRm8SkDvsycE+JkV3WGY41e+IM= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0 h1:hT8rVHwugYE2lEfdFE0QWVo81lF7jMrYJVDWI+f+VxU= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.0/go.mod h1:8tu/lYfQfFe6IGnaOdrpVgEL2IrrDOf6/m9RQum4NkY= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 h1:lMwCXiWJlrtZot0NJTjbC8G9zl+V3i68gBTBBvDeEXA= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2 h1:Ji0DY1xUsUr3I8cHps0G+XM3WWU16lP6yG8qu1GAZAs= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.2/go.mod h1:5CsjAbs3NlGQyZNFACh+zztPDI7fU6eW9QsxjfnuBKg= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3 h1:dT3MqvGhSoaIhRseqw2I0yH81l7wiR2vjs57O51EAm8= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.11.3/go.mod h1:GlAeCkHwugxdHaueRr4nhPuY+WW+gR8UjlcqzPr1SPI= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.11 h1:o4T+fKxA3gTMcluBNZZXE9DNaMkJuUL1O3mffCUjoJo= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.11/go.mod h1:84oZdJ+VjuJKs9v1UTC9NaodRZRseOXCTgku+vQJWR8= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17 h1:HGErhhrxZlQ044RiM+WdoZxp0p+EGM62y3L6pwA4olE= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.11.17/go.mod h1:RkZEx4l0EHYDJpWppMJ3nD9wZJAa8/0lq9aVC+r2UII= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -717,16 +717,16 @@ 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.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= 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.20.11 h1:gEYM2GSpr4YNWc6hCd5nod4+d4kd9vWIAWrmGuLdlMw= -github.com/aws/aws-sdk-go-v2/service/sso v1.20.11/go.mod h1:gVvwPdPNYehHSP9Rs7q27U1EU+3Or2ZpXvzAYJNh63w= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.5 h1:iXjh3uaH3vsVcnyZX7MqCoCfcyxIrVE9iOQruRaWPrQ= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.24.5/go.mod h1:5ZXesEuy/QcO0WUnt+4sDkxhdXRHTu2yG0uCSH8B6os= +github.com/aws/aws-sdk-go-v2/service/sso v1.22.4 h1:BXx0ZIxvrJdSgSvKTZ+yRBeSqqgPM89VPlulEcl37tM= +github.com/aws/aws-sdk-go-v2/service/sso v1.22.4/go.mod h1:ooyCOXjvJEsUw7x+ZDHeISPMhtwI3ZCB7ggFMcFfWLU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4 h1:yiwVzJW2ZxZTurVbYWA7QOrAaCYQR72t0wrSBfoesUE= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.26.4/go.mod h1:0oxfLkpz3rQ/CHlx5hB7H69YUpFiI1tql6Q6Ne+1bCw= 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.28.12 h1:M/1u4HBpwLuMtjlxuI2y6HoVLzF5e2mfxHCg7ZVMYmk= -github.com/aws/aws-sdk-go-v2/service/sts v1.28.12/go.mod h1:kcfd+eTdEi/40FIbLq4Hif3XMXnl5b/+t/KTfLt9xIk= +github.com/aws/aws-sdk-go-v2/service/sts v1.30.3 h1:ZsDKRLXGWHk8WdtyYMoGNO7bTudrvuKpDKgMVRlepGE= +github.com/aws/aws-sdk-go-v2/service/sts v1.30.3/go.mod h1:zwySh8fpFyXp9yOr/KVzxOl8SRqgf/IDw5aUt9UKFcQ= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.20.2 h1:tbp628ireGtzcHDDmLT/6ADHidqnwgF57XOXZe6tp4Q= -github.com/aws/smithy-go v1.20.2/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= +github.com/aws/smithy-go v1.20.3 h1:ryHwveWzPV5BIof6fyDvor6V3iUL7nTfiTKXHiW05nE= +github.com/aws/smithy-go v1.20.3/go.mod h1:krry+ya/rV9RDcV/Q16kpu6ypI4K2czasz0NC3qS14E= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= @@ -1003,7 +1003,6 @@ github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= -github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 h1:eQGUsj2LcsLzfrHY1noKDSU7h+c9/rw9pQPwbQ9g1jQ= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6/go.mod h1:LIAXxPvcUXwOcTIj9LSNSUpE9/eMHalTWxsP/kmWxQI=