[ 
https://issues.apache.org/jira/browse/HDDS-1333?focusedWorklogId=221810&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-221810
 ]

ASF GitHub Bot logged work on HDDS-1333:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Apr/19 16:06
            Start Date: 02/Apr/19 16:06
    Worklog Time Spent: 10m 
      Work Description: xiaoyuyao commented on pull request #653: HDDS-1333. 
OzoneFileSystem can't work with spark/hadoop2.7 because incompatible security 
classes
URL: https://github.com/apache/hadoop/pull/653#discussion_r271382877
 
 

 ##########
 File path: hadoop-hdds/docs/content/SparkOzoneFSK8S.md
 ##########
 @@ -78,11 +78,13 @@ And create a custom `core-site.xml`:
 <configuration>
     <property>
         <name>fs.o3fs.impl</name>
-        <value>org.apache.hadoop.fs.ozone.OzoneFileSystem</value>
+        <value>org.apache.hadoop.fs.ozone.BasicOzoneFileSystem</value>
     </property>
 </configuration>
 ```
 
+_Note_: You may also use `org.apache.hadoop.fs.ozone.OzoneFileSystem` without 
the `Basic` prefix. The `Basic` version doesn't support FS statistics and 
security tokens but can work together with older hadoop versions.
 
 Review comment:
   This is not accurate. If I understand correctly, the BasicOzoneFileSystem 
also support delegation token APIs but not FS statistics.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 221810)
    Time Spent: 4h 20m  (was: 4h 10m)

> OzoneFileSystem can't work with spark/hadoop2.7 because incompatible security 
> classes
> -------------------------------------------------------------------------------------
>
>                 Key: HDDS-1333
>                 URL: https://issues.apache.org/jira/browse/HDDS-1333
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>            Reporter: Elek, Marton
>            Assignee: Elek, Marton
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 4h 20m
>  Remaining Estimate: 0h
>
> The current ozonefs compatibility layer is broken by: HDDS-1299.
> The spark jobs (including hadoop 2.7) can't be executed any more:
> {code}
> 2019-03-25 09:50:08 INFO  StateStoreCoordinatorRef:54 - Registered 
> StateStoreCoordinator endpoint
> Exception in thread "main" java.lang.NoClassDefFoundError: 
> org/apache/hadoop/crypto/key/KeyProviderTokenIssuer
>         at java.lang.ClassLoader.defineClass1(Native Method)
>         at java.lang.ClassLoader.defineClass(ClassLoader.java:763)
>         at 
> java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
>         at java.net.URLClassLoader.defineClass(URLClassLoader.java:468)
>         at java.net.URLClassLoader.access$100(URLClassLoader.java:74)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:369)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findClass(URLClassLoader.java:362)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>         at java.lang.Class.forName0(Native Method)
>         at java.lang.Class.forName(Class.java:348)
>         at 
> org.apache.hadoop.conf.Configuration.getClassByNameOrNull(Configuration.java:2134)
>         at 
> org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2099)
>         at 
> org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2193)
>         at 
> org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2654)
>         at 
> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2667)
>         at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94)
>         at 
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703)
>         at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685)
>         at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373)
>         at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295)
>         at 
> org.apache.spark.sql.execution.streaming.FileStreamSink$.hasMetadata(FileStreamSink.scala:45)
>         at 
> org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:332)
>         at 
> org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:223)
>         at 
> org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:211)
>         at 
> org.apache.spark.sql.DataFrameReader.text(DataFrameReader.scala:715)
>         at 
> org.apache.spark.sql.DataFrameReader.textFile(DataFrameReader.scala:757)
>         at 
> org.apache.spark.sql.DataFrameReader.textFile(DataFrameReader.scala:724)
>         at org.apache.spark.examples.JavaWordCount.main(JavaWordCount.java:45)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at 
> org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
>         at 
> org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:849)
>         at 
> org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:167)
>         at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:195)
>         at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
>         at 
> org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:924)
>         at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:933)
>         at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> Caused by: java.lang.ClassNotFoundException: 
> org.apache.hadoop.crypto.key.KeyProviderTokenIssuer
>         at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>         ... 43 more
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to