Uploaded image for project: 'Apache Gobblin'
  1. Apache Gobblin
  2. GOBBLIN-1972

Fix Iceberg-Distcp, which now erroneously shows the job finishing as "SUCCESSFUL", despite the commit step failing with an exception.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • None
    • None
    • gobblin-core
    • None

    Description

      Find and fix what appears to be a regression in `CopyDataPublisher` or `SafeDatasetCommit`...

      Here, the same Iceberg-Distcp `CommitStep` fails (`org.apache.gobblin.data.management.copy.iceberg.IcebergRegisterStep`), but with apache gobblin build `0.18.0-dev-493` it correctly shows "FAILED" yet with `0.18.0-dev-515` it claims "SUCCEEDED".

      BOTH display the same error:

      ERROR [CopyDataPublisher] [Azkaban] Failed to publish tracking_db.MyEvent
        [then the exception stacktrace for the commit step, followed by...]
      
      ERROR [SafeDatasetCommit] [Azkaban] Failed to commit dataset
       - java.io.IOException: Not all datasets published successfully
       -     at org.apache.gobblin.data.management.copy.publisher.CopyDataPublisher.publishData(CopyDataPublisher.java:160)
       -     at org.apache.gobblin.publisher.DataPublisher.publish(DataPublisher.java:79)
       -     at org.apache.gobblin.runtime.SafeDatasetCommit.commitDataset(SafeDatasetCommit.java:260)
       -     at org.apache.gobblin.runtime.SafeDatasetCommit.call(SafeDatasetCommit.java:168)
        [...]

      but only "FAILED" continues with:

      WARN [SafeDatasetCommit] [Azkaban] At least one task in CopyEntity.DatasetAndPartition(dataset=CopyableDatasetMetadata(datasetURN=tracking_db.MyEvent), partition=tracking_db.MyEvent) did not get committed successfully. Setting dataset state to FAILED. java.io.IOException: Not all datasets published successfully
       -     at org.apache.gobblin.data.management.copy.publisher.CopyDataPublisher.publishData(CopyDataPublisher.java:160)
       -     at org.apache.gobblin.publisher.DataPublisher.publish(DataPublisher.java:79)
       -     at org.apache.gobblin.runtime.SafeDatasetCommit.commitDataset(SafeDatasetCommit.java:260)
       -     at org.apache.gobblin.runtime.SafeDatasetCommit.call(SafeDatasetCommit.java:168)
      [...]
      
      INFO [AbstractJobLauncher] [Azkaban] Due to task failure, will report that no records or bytes were written for CopyEntity.DatasetAndPartition(dataset=CopyableDatasetMetadata(datasetURN=tracking_db.MyEvent), partition=tracking_db.MyEvent)

      by contrast, "SUCCEEDED" announces:

      INFO [SafeDatasetCommit] [Azkaban] Persisting dataset state for dataset CopyEntity.DatasetAndPartition(dataset=CopyableDatasetMetadata(datasetURN=tracking_db.MyEvent), partition=tracking_db.MyEvent)
      [...]
      INFO [AbstractJobLauncher] [Azkaban] DatasetMetrics for 'CopyEntity.DatasetAndPartition(dataset=CopyableDatasetMetadata(datasetURN=tracking_db.MyEvent), partition=tracking_db.MyEvent)' - (records: 0; bytes: 0) 

      Attachments

        Activity

          People

            abti Abhishek Tiwari
            kipk Kip Kohn
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Time Tracking

                Estimated:
                Original Estimate - Not Specified
                Not Specified
                Remaining:
                Remaining Estimate - 0h
                0h
                Logged:
                Time Spent - 1h 10m
                1h 10m