Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Bug]: BigQuery IO Batch load using File_load causing the same job id ignoring inserts as the job_id is already completed #28219

Closed
1 of 15 tasks
yeshvantbhavnasi opened this issue Aug 30, 2023 · 22 comments

Comments

@yeshvantbhavnasi
Copy link

yeshvantbhavnasi commented Aug 30, 2023

What happened?

BigQueryIO.Write<Row> batchWrite =
          rowWrite
              .withMethod(BigQueryIO.Write.Method.FILE_LOADS)
              .withTriggeringFrequency(Duration.standardMinutes(fileLoadSpecOptional.get().writeTriggerFrequency))
              .withAutoSharding()
              .withExtendedErrorInfo()
              .withMaxRetryJobs(fileLoadSpecOptional.get().withJobRetryCount)
              .withWriteTempDataset(TMP_TABLE_DATASET)
              .withCustomGcsTempLocation(ValueProvider.StaticValueProvider.of(fileLoadSpecOptional.get().stagingBucket));

Have following code to setup file_load operation for Dataflow streaming jobs using GCS and File_notification based ingestion into BigQuery

  • It creates a JobId for new file after few minutes and load data into Live table with jobId beam_bq_job_LOAD_defaultnetworkvpcflowlogslive_e78561b1b6e147e7abafe83b9314c7f1_41dd2dbb7ce5b6e717e5aadb4244b4e0_00001_00000-0
    but after this every new files are added to staging bucket new set of files have exactly 1 partition and c.pane.index seems to be 0 causing the same job_id and ignoring file writes
    See the screenshot for different timestamp we have the same job_id ignoring inserts
image

Should we add random uuid to jobId logic to fix this issue here ?

by changing this line to:

 String jobIdPrefix =
          BigQueryResourceNaming.createJobIdWithDestination(
              c.sideInput(loadJobIdPrefixView), tableDestination, partition, c.pane().getIndex()) + "-" +  random5CharUUID();

Issue Priority

Priority: 2 (default / most bugs should be filed as P2)

Issue Components

  • Component: Python SDK
  • Component: Java SDK
  • Component: Go SDK
  • Component: Typescript SDK
  • Component: IO connector
  • Component: Beam examples
  • Component: Beam playground
  • Component: Beam katas
  • Component: Website
  • Component: Spark Runner
  • Component: Flink Runner
  • Component: Samza Runner
  • Component: Twister2 Runner
  • Component: Hazelcast Jet Runner
  • Component: Google Cloud Dataflow Runner
@maskalski-clgx
Copy link

We faced the same issue after we switched to Dataflow Runner v2. BigQuery Load job with the same jobId is used for every batch write so only the first batch is actually written.

In addition to @yeshvantbhavnasi 's logs, I have the following warnings:
image

@Abacn
Copy link
Contributor

Abacn commented Aug 30, 2023

Thanks for reporting this. It's clearly a bug and should be fixed.

We faced the same issue after we switched to Dataflow Runner v2.

This also suggests runner v2 somehow generates smaller load jobs. Most of time, within one window it should only have one load job. Another issue to track.

@Abacn
Copy link
Contributor

Abacn commented Aug 30, 2023

I see, they are different window. Should also add window to job name.

@Abacn Abacn self-assigned this Aug 30, 2023
@Abacn
Copy link
Contributor

Abacn commented Aug 30, 2023

There is a bug in Dataflow Runner V2 where ProcessContext c.pane().getIndex() always returns 0 here:

c.sideInput(loadJobIdPrefixView), tableDestination, partition, c.pane().getIndex());

Causing duplicated BigQuery job id. However in Dataflow Runner V1 the pane index is increasing.

@Abacn
Copy link
Contributor

Abacn commented Aug 30, 2023

Either use legacy runner (with streaming engine enabled); or under Dataflow runner v2 and manually set sharding BigQueryIO.write().withNumFileShards(1) or any positive integer would mitigate this issue.

@yeshvantbhavnasi
Copy link
Author

Either use legacy runner (with streaming engine enabled); or under Dataflow runner v2 and manually set sharding BigQueryIO.write().withNumFileShards(1) or any positive integer would mitigate this issue.

I did try using withNumFileShards still see the same problem causing the same load job-id

@Abacn
Copy link
Contributor

Abacn commented Aug 31, 2023

Either use legacy runner (with streaming engine enabled); or under Dataflow runner v2 and manually set sharding BigQueryIO.write().withNumFileShards(1) or any positive integer would mitigate this issue.

I did try using withNumFileShards still see the same problem causing the same load job-id

You're right, sorry, I messed up different test pipelines. Confirm that setting withNumFileShards issue still exists. The problem is not need GroupIntoBatches to trigger

@Abacn
Copy link
Contributor

Abacn commented Aug 31, 2023

The problem is Downstream PCollection lost the pane info of first GBK after a ReShuffle. The pipeline involved is: Trigger->GBK->ReShuffle->Downstream

ReShuffle happens here:

After discussion we think the pane index is either work as intended or undefined behavior. While GBK will re-fire a pane, ReShuffle is implemented differently in runners. Current behavior is inconsistent among runner:

Is pane info preserved in Downstream for a pipeline with Trigger->GBK->ReShuffle->Downstream?

runner ReShuffle.of() ReShuffle.viaRandomKey
Java Direct Runner Yes No
Java Dataflow Legacy Runner Yes Yes
Java Dataflow Runner V2 No No

Opened #28272 for a fix. Added randomness to job id.

The implication of randomness in job id is that if the bundle gets retried it will have duplicates.

Note that Python SDK implementation did not consider pane info but also uses random job id:

@Abacn Abacn added this to the 2.51.0 Release milestone Sep 2, 2023
@kennknowles
Copy link
Member

I think there may be too much trial-and-error guessing here. We can reason about this and get it right.

When you have an aggregation (either a GBK or a Combine) then the elements coming out of that aggregation are uniquely identified by key + window + pane index. The pane index is how you can tell the difference between different triggerings.

A "Reshuffle" has a trigger that always fires as fast as possible. The pane index should increase with each element. But if it is by random key then the actual key that was shuffled by is dropped so the pane index doesn't mean anything any more.

Now the problem is that Reshuffle also breaks apart the GBK result into individual elements again. There is no guarantee that these elements will always travel together in a bundle, so again the pane index is not a reliable thing.

@Abacn Abacn removed this from the 2.51.0 Release milestone Sep 26, 2023
@Abacn
Copy link
Contributor

Abacn commented Sep 26, 2023

removed 2.51.0 milestone as a consensus is pending

@kennknowles
Copy link
Member

Question: if we do assume the behavior should be like I said in the design doc and like Reuven said - the exact identity function - would this change be needed or not?

@Abacn
Copy link
Contributor

Abacn commented Sep 28, 2023

Question: if we do assume the behavior should be like I said in the design doc and like Reuven said - the exact identity function - would this change be needed or not?

This change is then not needed

@yeshvantbhavnasi
Copy link
Author

Can we check if this can be prioritized ?
We want to use batch-load for our use-cases which will reduce the price on streaming api.

@kennknowles
Copy link
Member

For Dataflow v1 the behavior is already correct.

@yeshvantbhavnasi
Copy link
Author

@kennknowles we cannot use dataflow v1 as we are using custom images for security which is supported only on dataflow v2

@kennknowles
Copy link
Member

Got it. This is a high priority. It is underway but takes some work. (fixing it in the SDK does not necessarily fix it for all the runners, because they do customized things under the hood)

@kennknowles
Copy link
Member

To be clear: the proposed change actually would be a different behavior that we do not desire. The work here is to fix Reshuffle implementations.

@kennknowles kennknowles assigned kennknowles and unassigned Abacn Oct 11, 2023
@yeshvantbhavnasi
Copy link
Author

yeshvantbhavnasi commented Nov 7, 2023

Hey @kennknowles can we point to reshuffle implementation codebase ?
We are looking if we can try rollout our own version of beam after the fix and see if it works on GCP Dataflow with batch .

@kennknowles
Copy link
Member

Ah, sorry I missed this message. This change is #28853 which is in Beam 2.54.0 and above.

@github-actions github-actions bot added this to the 2.55.0 Release milestone Feb 28, 2024
@pranjal5215
Copy link

pranjal5215 commented Oct 19, 2024

Hi @kennknowles @Abacn ,

I am using Batch load job (moving data from GCS- Cloud Storage to BigQuery) , and Job shows successfully run, but writes only a small portion of data to BigQuery table, Typically 2- 10 rows out of 1071 (expected),
While searching more, I stumbled upon above discussion here. From diagnostics it seems to be similar issue , Getting a lot of 409s at bigquery write step (screenshot attached) , Also most BigQuery jobs seems to have same IDs (attached screenshot)

Is this the same issue,
Happy to share more details if required on this

image image

Another version of this job with exact same DynamicDestination configuration is running successfully with streaming setup - (Basically that pipeline is moving data from Kafka to BigQuery)

Java SDK version:
2.60.0
2.57.0
2.53.0 (Because above it mentiones that bug was intriduced on 2.54.0 and above ?)

Runner:
GCP Dataflow

Tried to run with both below- but job still writes only partial output to BQ

  1. Dataflow runner v2
  2. Disabling runner v2

JobID looks like:
beam_bq_job_LOAD_debugbqload_952f816652304fca8d830ed781542535_3739ddb1bbdb83c1edfec375a931a79a_00001_00000-0
Have given job name like below

        options.setJobName("debug-bq-load");
        WriteResult result = bigQueryRows.apply("WriteToBigQuery",
                BigQueryIO.<BigQueryRow>write()
                        .to(new MyDynamicDestination(bigqueryDatasetName))
                        .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
                        .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_EMPTY)
                        .withFormatFunction(bigQueryRow -> bigQueryRow != null ? bigQueryRow.getFormattedTableRow() : null)
                        .withMethod(BigQueryIO.Write.Method.FILE_LOADS)
        );

Also pasting dataflow job describe output here

gcloud dataflow jobs describe 2024-10-18_11_41_06-2506487362125660927 --region=asia-south1  --format=json
{
  "createTime": "2024-10-18T18:41:07.476926Z",
  "currentState": "JOB_STATE_DONE",
  "currentStateTime": "2024-10-18T18:44:52.815102Z",
  "environment": {
    "shuffleMode": "SERVICE_BASED",
    "userAgent": {
      "container.base_repository": "gcr.io/cloud-dataflow/v1beta3",
      "fnapi.container.version": "2.60.0",
      "fnapi.environment.major.version": "8",
      "java.vendor": "Oracle Corporation",
      "java.version": "18.0.1.1",
      "legacy.container.version": "2.60.0",
      "legacy.environment.major.version": "8",
      "name": "Apache Beam SDK for Java",
      "os.arch": "aarch64",
      "os.name": "Mac OS X",
      "os.version": "14.6.1",
      "version": "2.60.0"
    },
    "version": {
      "job_type": "JAVA_BATCH_AUTOSCALING",
      "major": "8"
    }
  },
  "id": "2024-10-18_11_41_06-2506487362125660927",
  "jobMetadata": {
    "sdkVersion": {
      "sdkSupportStatus": "SUPPORTED",
      "version": "2.60.0",
      "versionDisplayName": "Apache Beam SDK for Java"
    }
  },
  "location": "asia-south1",
  "name": "gcs_to_bq",
  "projectId": "livewel-prod",
  "stageStates": [
    {
      "currentStateTime": "2024-10-18T18:43:34.734Z",
      "executionStageName": "s74.org.apache.beam.sdk.values.PCollection.<init>:397#bed3177b52ae68e-c181",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:34.922Z",
      "executionStageName": "s75.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:18.147Z",
      "executionStageName": "F204",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.589Z",
      "executionStageName": "s81-close-shuffle148",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:37.382Z",
      "executionStageName": "s109-close-shuffle51",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.439Z",
      "executionStageName": "s54-shuffle-session105",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.618Z",
      "executionStageName": "s32-shuffle-session77",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.770Z",
      "executionStageName": "s2840-shuffle-session71",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.954Z",
      "executionStageName": "s109-shuffle-session48",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.719Z",
      "executionStageName": "s3913-shuffle-session85",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.589Z",
      "executionStageName": "success211",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.996Z",
      "executionStageName": "F185",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.037Z",
      "executionStageName": "s92-open-shuffle164",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:19.508Z",
      "executionStageName": "F198",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:17.417Z",
      "executionStageName": "s22.org.apache.beam.sdk.values.PCollection.<init>:397#1d275f544daf228c",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:13.890Z",
      "executionStageName": "F187",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:17.380Z",
      "executionStageName": "F205",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.819Z",
      "executionStageName": "s45",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.204Z",
      "executionStageName": "F207",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.732Z",
      "executionStageName": "s504-shuffle-session99",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.097Z",
      "executionStageName": "s43-close-shuffle94",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.513Z",
      "executionStageName": "s43-open-shuffle92",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.061Z",
      "executionStageName": "s92-shuffle-session163",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.734Z",
      "executionStageName": "s6-close-shuffle116",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.694Z",
      "executionStageName": "s86-open-shuffle155",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.554Z",
      "executionStageName": "s43-shuffle-session91",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:34.894Z",
      "executionStageName": "s75",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:38.225Z",
      "executionStageName": "s112.org.apache.beam.sdk.values.PCollection.<init>:397#689338c1ae4b4531",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.096Z",
      "executionStageName": "s86-close-shuffle157",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:17.707Z",
      "executionStageName": "s6122-shuffle-session122",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:17.670Z",
      "executionStageName": "s6122-open-shuffle123",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.145Z",
      "executionStageName": "s113.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.713Z",
      "executionStageName": "F195",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.764Z",
      "executionStageName": "s55.org.apache.beam.sdk.values.PCollection.<init>:397#30b23d9769235bb2",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.683Z",
      "executionStageName": "s97-open-shuffle173",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.755Z",
      "executionStageName": "s1731-shuffle-session57",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.059Z",
      "executionStageName": "s81-open-shuffle146",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.874Z",
      "executionStageName": "s56.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:34.809Z",
      "executionStageName": "s73-u182",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.734Z",
      "executionStageName": "s21-open-shuffle64",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.717Z",
      "executionStageName": "F192",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.086Z",
      "executionStageName": "s54-close-shuffle108",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.475Z",
      "executionStageName": "F200",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:18.662Z",
      "executionStageName": "s65-close-shuffle131",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.697Z",
      "executionStageName": "s1731-open-shuffle58",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.384Z",
      "executionStageName": "s3913-close-shuffle88",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.832Z",
      "executionStageName": "s56",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.524Z",
      "executionStageName": "F210",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:34.837Z",
      "executionStageName": "s74.org.apache.beam.sdk.values.PCollection.<init>:397#bed3177b52ae68e",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.015Z",
      "executionStageName": "F188",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:27.777Z",
      "executionStageName": "F202",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.561Z",
      "executionStageName": "F186",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.824Z",
      "executionStageName": "s101.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.697Z",
      "executionStageName": "s504-open-shuffle100",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.782Z",
      "executionStageName": "s97-close-shuffle175",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:21.444Z",
      "executionStageName": "start213",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.696Z",
      "executionStageName": "s6-open-shuffle114",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.351Z",
      "executionStageName": "F209",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.698Z",
      "executionStageName": "s2840-open-shuffle72",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.514Z",
      "executionStageName": "s33.org.apache.beam.sdk.values.PCollection.<init>:397#c22b25ea2180c33a",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.119Z",
      "executionStageName": "s113",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.084Z",
      "executionStageName": "s81-shuffle-session145",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.750Z",
      "executionStageName": "s44.org.apache.beam.sdk.values.PCollection.<init>:397#b06373aead4e1949",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:44:03.052Z",
      "executionStageName": "s90.org.apache.beam.sdk.values.PCollection.<init>:397#eb37685a4ffcf182",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:18.378Z",
      "executionStageName": "s65-shuffle-session128",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:19.613Z",
      "executionStageName": "s67",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.721Z",
      "executionStageName": "s86-shuffle-session154",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.433Z",
      "executionStageName": "s2840-close-shuffle74",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.281Z",
      "executionStageName": "s504-close-shuffle102",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:18.590Z",
      "executionStageName": "F199",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.003Z",
      "executionStageName": "F193",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.857Z",
      "executionStageName": "s45.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:27.814Z",
      "executionStageName": "s74.org.apache.beam.sdk.values.PCollection.<init>:397#bed3177b52ae68e-c180",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.304Z",
      "executionStageName": "F208",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.741Z",
      "executionStageName": "s100.org.apache.beam.sdk.values.PCollection.<init>:397#160896c2d158da1f",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.692Z",
      "executionStageName": "s3913-open-shuffle86",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.564Z",
      "executionStageName": "s32-open-shuffle78",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:19.778Z",
      "executionStageName": "s70-open-shuffle137",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.163Z",
      "executionStageName": "F201",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.161Z",
      "executionStageName": "F206",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.797Z",
      "executionStageName": "s101",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:19.813Z",
      "executionStageName": "s70-shuffle-session136",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:38.199Z",
      "executionStageName": "F189",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:21.363Z",
      "executionStageName": "failure212",
      "executionStageState": "JOB_STATE_PENDING"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.245Z",
      "executionStageName": "s32-close-shuffle80",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:17.533Z",
      "executionStageName": "s23.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.729Z",
      "executionStageName": "F190",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.608Z",
      "executionStageName": "s1731-close-shuffle60",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.003Z",
      "executionStageName": "F191",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.573Z",
      "executionStageName": "s92-close-shuffle166",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.526Z",
      "executionStageName": "s21-close-shuffle66",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:14.396Z",
      "executionStageName": "s54-open-shuffle106",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.709Z",
      "executionStageName": "s97-shuffle-session172",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:37.322Z",
      "executionStageName": "F194",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:17.494Z",
      "executionStageName": "s23",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:19.543Z",
      "executionStageName": "s66.org.apache.beam.sdk.values.PCollection.<init>:397#556f4ebf84d9869b",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:27.864Z",
      "executionStageName": "s70-close-shuffle139",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:35.511Z",
      "executionStageName": "F196",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.773Z",
      "executionStageName": "s21-shuffle-session63",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:19.647Z",
      "executionStageName": "s67.output",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:41:30.743Z",
      "executionStageName": "s6-shuffle-session113",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.584Z",
      "executionStageName": "s34",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:36.929Z",
      "executionStageName": "s109-open-shuffle49",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:18.216Z",
      "executionStageName": "s6122-close-shuffle125",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:34.708Z",
      "executionStageName": "F197",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:18.341Z",
      "executionStageName": "s65-open-shuffle129",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:15.653Z",
      "executionStageName": "F203",
      "executionStageState": "JOB_STATE_DONE"
    },
    {
      "currentStateTime": "2024-10-18T18:43:16.621Z",
      "executionStageName": "s34.output",
      "executionStageState": "JOB_STATE_DONE"
    }
  ],
  "startTime": "2024-10-18T18:41:07.476926Z",
  "type": "JOB_TYPE_BATCH"
}

@priyansndesai
Copy link
Contributor

@pranjal5215 are you experiencing this with both - with and without V2?

@liferoad
Copy link
Collaborator

Possible to share the code here?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

7 participants