Uploaded image for project: 'Apache Hudi'
  1. Apache Hudi
  2. HUDI-7390

[Regression] HoodieStreamer no longer works without --props being supplied

    XMLWordPrintableJSON

Details

    Description

      Problem

      When attempting to run HoodieStreamer without a props file, specifying all required extra configuration via --hoodie-conf parameters, the execution fails and an exception is thrown:

      24/02/06 22:15:13 INFO SparkContext: Successfully stopped SparkContext
      Exception in thread "main" org.apache.hudi.exception.HoodieIOException: Cannot read properties from dfs from file file:/private/tmp/hudi-props-repro/src/test/resources/streamer-config/dfs-source.properties
              at org.apache.hudi.common.config.DFSPropertiesConfiguration.addPropsFromFile(DFSPropertiesConfiguration.java:166)
              at org.apache.hudi.common.config.DFSPropertiesConfiguration.<init>(DFSPropertiesConfiguration.java:85)
              at org.apache.hudi.utilities.UtilHelpers.readConfig(UtilHelpers.java:232)
              at org.apache.hudi.utilities.streamer.HoodieStreamer$Config.getProps(HoodieStreamer.java:437)
              at org.apache.hudi.utilities.streamer.StreamSync.getDeducedSchemaProvider(StreamSync.java:656)
              at org.apache.hudi.utilities.streamer.StreamSync.fetchNextBatchFromSource(StreamSync.java:632)
              at org.apache.hudi.utilities.streamer.StreamSync.fetchFromSourceAndPrepareRecords(StreamSync.java:525)
              at org.apache.hudi.utilities.streamer.StreamSync.readFromSource(StreamSync.java:498)
              at org.apache.hudi.utilities.streamer.StreamSync.syncOnce(StreamSync.java:404)
              at org.apache.hudi.utilities.streamer.HoodieStreamer$StreamSyncService.ingestOnce(HoodieStreamer.java:850)
              at org.apache.hudi.utilities.ingestion.HoodieIngestionService.startIngestion(HoodieIngestionService.java:72)
              at org.apache.hudi.common.util.Option.ifPresent(Option.java:97)
              at org.apache.hudi.utilities.streamer.HoodieStreamer.sync(HoodieStreamer.java:207)
              at org.apache.hudi.utilities.streamer.HoodieStreamer.main(HoodieStreamer.java:592)
              at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
              at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
              at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.base/java.lang.reflect.Method.invoke(Method.java:568)
              at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
              at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1020)
              at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:192)
              at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:215)
              at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
              at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1111)
              at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1120)
              at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
      Caused by: java.io.FileNotFoundException: File file:/private/tmp/hudi-props-repro/src/test/resources/streamer-config/dfs-source.properties does not exist
              at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:779)
              at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:1100)
              at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:769)
              at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:462)
              at org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.<init>(ChecksumFileSystem.java:160)
              at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:372)
              at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:976)
              at org.apache.hudi.common.config.DFSPropertiesConfiguration.addPropsFromFile(DFSPropertiesConfiguration.java:161)
              ... 25 more 

      Reproduction Steps

      1. Setup clean spark install

      mkdir /tmp/hudi-props-repro 
      cd /tmp/hudi-props-repro 
      tar -xvzf ~/spark-3.4.2-bin-hadoop3.tgz

      2. Copy the schema file from the docker demo

      wget https://raw.githubusercontent.com/apache/hudi/release-0.14.1/docker/demo/config/schema.avsc 

      3. Copy data file from the docker demo

      mkdir data
      cd data
      wget https://raw.githubusercontent.com/apache/hudi/release-0.14.1/docker/demo/data/batch_1.json 
      cd .. 

      4. Run HoodieStreamer

      spark-3.4.2-bin-hadoop3/bin/spark-submit \
         --packages org.apache.hudi:hudi-utilities-slim-bundle_2.12:0.14.1,org.apache.hudi:hudi-spark3.4-bundle_2.12:0.14.1 \
         --conf spark.kryoserializer.buffer.max=200m \
         --conf spark.serializer=org.apache.spark.serializer.KryoSerializer \
         --class org.apache.hudi.utilities.streamer.HoodieStreamer \
         spark-3.4.2-bin-hadoop3/examples/jars/spark-examples_2.12-3.4.2.jar \
         --table-type COPY_ON_WRITE \
         --source-class org.apache.hudi.utilities.sources.JsonDFSSource \
         --target-base-path /tmp/hudi-props-repro/table \
         --target-table table \
         --hoodie-conf hoodie.datasource.write.recordkey.field=key \
         --hoodie-conf hoodie.datasource.write.partitionpath.field=date \
         --hoodie-conf hoodie.table.recordkey.fields=key \
         --hoodie-conf hoodie.table.partition.fields=date \
         --hoodie-conf hoodie.streamer.schemaprovider.source.schema.file=/tmp/hudi-props-repro/schema.avsc \
         --hoodie-conf hoodie.streamer.schemaprovider.target.schema.file=/tmp/hudi-props-repro/schema.avsc \
         --hoodie-conf hoodie.streamer.source.dfs.root=/tmp/hudi-props-repro/data \
         --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider
      

      Expected Results

      Command runs successfully, data is ingested successfully into /tmp/hudi-decimal-repro/table, some files exist under /tmp/hudi-decimal-repro/table/2018/08/31/.

      Actual Results

      Command fails with exception, no data is ingsted into the table.  The table's directory is initialized but no commits exist.

      Logs of the attempted run are attached as spark.log

      Additional Information

      This issue does not appear to exist in versions 0.12.2 through 0.14.0 based on my own testing.  It does affect both the 0.14.1 and 1.0.0-beta1 releases.

      Known Workarounds

      You should be able to pass --props referencing either an empty file or even /dev/null to work around this issue.  Passing an empty string or a reference to a non-existent file will not work.

      Attachments

        1. spark.log
          163 kB
          Brandon Dahler

        Issue Links

          Activity

            People

              wombatukun Vova Kolmakov
              brandon.dahler.amazon Brandon Dahler
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: