Uploaded image for project: 'ORC'
  1. ORC
  2. ORC-1029

Could not load 'org.apache.orc.DataMask.Provider' when using orc encryption and spark executor with multi cores!

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Blocker
    • Resolution: Fixed
    • 1.7.0, 1.6.11
    • 1.7.1, 1.6.12
    • Java
    • None
    • jdk1.8
      spark 2.4.7
      orc 1.6.11
    • Important

    Description

      1.line 140 code in DataMask

          public static DataMask build(DataMaskDescription mask,
          TypeDescription schema,
          MaskOverrides overrides) {
          Iterator<Provider> iterator = LOADER.iterator();
          LOG.info("hgs-"Thread.currentThread().getName()"--start........" +mask.getName());
          LOG.info("hgs--"Thread.currentThread().getName()"—start........");

      140   while(iterator.hasNext())

      {     LOG.info("hgs--"+Thread.currentThread().getName()+"--"+ iterator.next().getClass().getName());    }

          LOG.info("hgs-"Thread.currentThread().getName()"--end........");
          for(Provider provider: LOADER) {
          DataMask result = provider.build(mask, schema, overrides);
          if (result != null)

      {     return result;     }

          }
          throw new IllegalArgumentException("Can't find data mask - " + mask);
          }

       

       

      2. the stacktrace

         21/10/19 17:30:35 ERROR executor.Executor: Exception in task 12.0 in stage 0.0 (TID 12)
      java.util.NoSuchElementException
      at sun.misc.CompoundEnumeration.nextElement(CompoundEnumeration.java:59)
      at java.util.ServiceLoader$LazyIterator.hasNextService(ServiceLoader.java:357)
      at java.util.ServiceLoader$LazyIterator.hasNext(ServiceLoader.java:393)
      at java.util.ServiceLoader$1.hasNext(ServiceLoader.java:474)
      at org.apache.orc.DataMask$Factory.build(DataMask.java:140)
      at org.apache.orc.impl.WriterImpl$StreamFactory.getUnencryptedMask(WriterImpl.java:443)
      at org.apache.orc.impl.writer.EncryptionTreeWriter.<init>(EncryptionTreeWriter.java:61)
      at org.apache.orc.impl.writer.TreeWriter$Factory.create(TreeWriter.java:130)
      at org.apache.orc.impl.writer.StructTreeWriter.<init>(StructTreeWriter.java:41)
      at org.apache.orc.impl.writer.TreeWriter$Factory.createSubtree(TreeWriter.java:181)
      at org.apache.orc.impl.writer.TreeWriter$Factory.create(TreeWriter.java:133)
      at org.apache.orc.impl.WriterImpl.<init>(WriterImpl.java:216)
      at org.apache.orc.OrcFile.createWriter(OrcFile.java:1008)
      at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.<init>(OrcOutputWriter.scala:48)
      at org.apache.spark.sql.execution.datasources.orc.OrcFileFormat$$anon$1.newInstance(OrcFileFormat.scala:123)
      at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:156)
      at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:142)
      at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:276)
      at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:210)
      at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:209)
      at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      at org.apache.spark.scheduler.Task.run(Task.scala:123)
      at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
      at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      at java.lang.Thread.run(Thread.java:745)
      21/10/19 17:30:35 ERROR executor.Executor: Exception in task 14.0 in stage 0.0 (TID 14)

       

       

      3. with one core is OK

      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........nullify
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7-org.apache.orc.impl.mask.SparkOrcMaskProvider
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7-org.apache.orc.impl.mask.MaskProvider
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--end........
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........nullify
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7-org.apache.orc.impl.mask.SparkOrcMaskProvider
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7-org.apache.orc.impl.mask.MaskProvider
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--end........
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........nullify
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7-org.apache.orc.impl.mask.SparkOrcMaskProvider
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7-org.apache.orc.impl.mask.MaskProvider
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--end........
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........nullify
      21/10/19 18:08:17 INFO orc.DataMask$Factory: hgs-Executor task launch worker for task 7--start........

      Attachments

        Issue Links

          Activity

            People

              Guiyankuang Yiqun Zhang
              hgs19921112 hgs
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: