This is an automated email from the ASF dual-hosted git repository. yhu pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push: new 1c63a0290fe Remove cloud debugger support (#25960) 1c63a0290fe is described below commit 1c63a0290fe139b2400f4e99e9044ce9269eeac8 Author: Yi Hu <ya...@google.com> AuthorDate: Tue Apr 18 16:44:37 2023 -0400 Remove cloud debugger support (#25960) * Remove cloud debugger pipeline option * Eliminated cloud-debugger api dependency --- CHANGES.md | 1 + .../org/apache/beam/gradle/BeamModulePlugin.groovy | 1 - runners/google-cloud-dataflow-java/build.gradle | 1 - .../beam/runners/dataflow/DataflowRunner.java | 61 --------------------- .../dataflow/options/CloudDebuggerOptions.java | 62 ---------------------- .../dataflow/options/DataflowPipelineOptions.java | 1 - .../runners/dataflow/util/DataflowTransport.java | 11 ---- 7 files changed, 1 insertion(+), 137 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 7be9752517d..0c21c895a08 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -69,6 +69,7 @@ ## Breaking Changes * X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* CloudDebuggerOptions is removed (deprecated in Beam v2.47.0) for Dataflow runner as the Google Cloud Debugger service is [shutting down](https://cloud.google.com/debugger/docs/deprecations). (Java) ([#25959](https://github.com/apache/beam/issues/25959)). ## Deprecations diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 60f01846709..22dda7af9e6 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -651,7 +651,6 @@ class BeamModulePlugin implements Plugin<Project> { google_api_client_java6 : "com.google.api-client:google-api-client-java6:$google_clients_version", google_api_common : "com.google.api:api-common", // google_cloud_platform_libraries_bom sets version google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20220924-$google_clients_version", - google_api_services_clouddebugger : "com.google.apis:google-api-services-clouddebugger:v2-rev20220318-$google_clients_version", google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20220828-$google_clients_version", google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20220920-$google_clients_version", google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20220818-$google_clients_version", diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 517249de76f..63fee049254 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -96,7 +96,6 @@ dependencies { // Ensures SequencedMessage availability for Spotless implementation library.java.proto_google_cloud_pubsublite_v1 permitUnusedDeclared library.java.proto_google_cloud_pubsublite_v1 - implementation library.java.google_api_services_clouddebugger implementation library.java.google_api_services_dataflow implementation library.java.google_api_services_storage permitUnusedDeclared library.java.google_api_services_storage // BEAM-11761 diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 5f9206c9e9d..2cc476dbe3c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -30,10 +30,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Stri import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.googleapis.json.GoogleJsonResponseException; -import com.google.api.services.clouddebugger.v2.CloudDebugger; -import com.google.api.services.clouddebugger.v2.model.Debuggee; -import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeRequest; -import com.google.api.services.clouddebugger.v2.model.RegisterDebuggeeResponse; import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; @@ -84,7 +80,6 @@ import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; import org.apache.beam.runners.dataflow.util.DataflowTemplateJob; -import org.apache.beam.runners.dataflow.util.DataflowTransport; import org.apache.beam.runners.dataflow.util.MonitoringUtil; import org.apache.beam.runners.dataflow.util.PackageUtil.StagedFile; import org.apache.beam.runners.dataflow.util.PropertyNames; @@ -826,58 +821,6 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> { } } - private String debuggerMessage(String projectId, String uniquifier) { - return String.format( - "To debug your job, visit Google Cloud Debugger at: " - + "https://console.developers.google.com/debug?project=%s&dbgee=%s", - projectId, uniquifier); - } - - private void maybeRegisterDebuggee(DataflowPipelineOptions options, String uniquifier) { - if (!options.getEnableCloudDebugger()) { - return; - } - - if (options.getDebuggee() != null) { - throw new RuntimeException("Should not specify the debuggee"); - } - - LOG.warn( - "The cloud debugger service is deprecated and will be turned down. Please remove " - + "`enableCloudDebugger` pipeline option as it will be removed in a future version."); - - CloudDebugger debuggerClient = DataflowTransport.newClouddebuggerClient(options).build(); - Debuggee debuggee = registerDebuggee(debuggerClient, uniquifier); - options.setDebuggee(debuggee); - - LOG.info(debuggerMessage(options.getProject(), debuggee.getUniquifier())); - } - - private Debuggee registerDebuggee(CloudDebugger debuggerClient, String uniquifier) { - RegisterDebuggeeRequest registerReq = new RegisterDebuggeeRequest(); - registerReq.setDebuggee( - new Debuggee() - .setProject(options.getProject()) - .setUniquifier(uniquifier) - .setDescription(uniquifier) - .setAgentVersion("google.com/cloud-dataflow-java/v1")); - - try { - RegisterDebuggeeResponse registerResponse = - debuggerClient.controller().debuggees().register(registerReq).execute(); - Debuggee debuggee = registerResponse.getDebuggee(); - if (debuggee.getStatus() != null && debuggee.getStatus().getIsError()) { - throw new RuntimeException( - "Unable to register with the debugger: " - + debuggee.getStatus().getDescription().getFormat()); - } - - return debuggee; - } catch (IOException e) { - throw new RuntimeException("Unable to register with the debugger: ", e); - } - } - protected RunnerApi.Pipeline applySdkEnvironmentOverrides( RunnerApi.Pipeline pipeline, DataflowPipelineOptions options) { String sdkHarnessContainerImageOverrides = options.getSdkHarnessContainerImageOverrides(); @@ -1225,10 +1168,6 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> { + "_" + randomNum; - // Try to create a debuggee ID. This must happen before the job is translated since it may - // update the options. - maybeRegisterDebuggee(dataflowOptions, requestId); - JobSpecification jobSpecification = translator.translate( pipeline, dataflowV1PipelineProto, dataflowV1Components, this, packages); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java deleted file mode 100644 index 8ddfbf053cd..00000000000 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.dataflow.options; - -import com.google.api.services.clouddebugger.v2.model.Debuggee; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.Hidden; -import org.apache.beam.sdk.options.PipelineOptions; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Options for controlling Cloud Debugger. - * - * @deprecated The cloud debugger service is turning down (<a - * href="https://cloud.google.com/debugger/docs/deprecations">announcement</a>). Cloud Debugger - * support and its options are deprecated in Beam and will be removed in Beam v2.48.0. - */ -@Description("[Deprecated] Used to configure the Cloud Debugger") -@Deprecated -@Hidden -public interface CloudDebuggerOptions extends PipelineOptions { - - /** Whether to enable the Cloud Debugger snapshot agent for the current job. */ - @Description("Whether to enable the Cloud Debugger snapshot agent for the current job.") - boolean getEnableCloudDebugger(); - - void setEnableCloudDebugger(boolean enabled); - - /** The Cloud Debugger debuggee to associate with. This should not be set directly. */ - @Description("The Cloud Debugger debuggee to associate with. This should not be set directly.") - @Hidden - @Nullable - Debuggee getDebuggee(); - - void setDebuggee(Debuggee debuggee); - - /** The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. */ - @Description( - "The maximum cost (as a ratio of CPU time) allowed for evaluating conditional snapshots. " - + "Should be a double between 0 and 1. " - + "Snapshots will be cancelled if evaluating conditions takes more than this ratio of time.") - @Default.Double(0.01) - double getMaxConditionCost(); - - void setMaxConditionCost(double maxConditionCost); -} diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java index aa09db9740f..c716cfe82ec 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java @@ -51,7 +51,6 @@ public interface DataflowPipelineOptions BigQueryOptions, GcsOptions, StreamingOptions, - CloudDebuggerOptions, DataflowWorkerLoggingOptions, DataflowProfilingOptions, PubsubOptions { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java index 3503b9588c5..e76bb134929 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java @@ -21,7 +21,6 @@ import static org.apache.beam.sdk.extensions.gcp.util.Transport.getJsonFactory; import static org.apache.beam.sdk.extensions.gcp.util.Transport.getTransport; import com.google.api.client.http.HttpRequestInitializer; -import com.google.api.services.clouddebugger.v2.CloudDebugger; import com.google.api.services.dataflow.Dataflow; import com.google.auth.Credentials; import com.google.auth.http.HttpCredentialsAdapter; @@ -84,16 +83,6 @@ public class DataflowTransport { .setGoogleClientRequestInitializer(options.getGoogleApiTrace()); } - public static CloudDebugger.Builder newClouddebuggerClient(DataflowPipelineOptions options) { - return new CloudDebugger.Builder( - getTransport(), - getJsonFactory(), - chainHttpRequestInitializer( - options.getGcpCredential(), new RetryHttpRequestInitializer())) - .setApplicationName(options.getAppName()) - .setGoogleClientRequestInitializer(options.getGoogleApiTrace()); - } - private static HttpRequestInitializer chainHttpRequestInitializer( Credentials credential, HttpRequestInitializer httpRequestInitializer) { if (credential == null) {