[
https://issues.apache.org/jira/browse/HDDS-1333?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16800842#comment-16800842
]
Elek, Marton commented on HDDS-1333:
------------------------------------
Yes, we should discuss again the effort to support older hadoop2 branches.
I have an idea to create a workaround: I created a simplified version of the
OzoneFileSystem:
{code}
public class OzoneFileSystemSimplified extends FileSystem {
//all the existing implementation except the file token magic
}
{code}
And the real file system is just an extension:
{code}
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class OzoneFileSystem extends OzoneFileSystemSimplified
implements KeyProviderTokenIssuer {
@Override
public KeyProvider getKeyProvider() throws IOException {
return getAdapter().getKeyProvider();
}
@Override
public URI getKeyProviderUri() throws IOException {
return getAdapter().getKeyProviderUri();
}
@Override
public DelegationTokenIssuer[] getAdditionalTokenIssuers()
throws IOException {
KeyProvider keyProvider = getKeyProvider();
if (keyProvider instanceof DelegationTokenIssuer) {
return new DelegationTokenIssuer[]{(DelegationTokenIssuer)keyProvider};
}
return null;
}
}
{code}
As of now I am testing this change, but I have some other problem
(`java.lang.NoClassDefFoundError: org/apache/hadoop/fs/StorageStatistics`)
which should be solved to test everything all together.
Still working on this.
> 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
> Priority: Major
>
> 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: [email protected]
For additional commands, e-mail: [email protected]