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

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

                Author: ASF GitHub Bot
            Created on: 01/Apr/19 10:49
            Start Date: 01/Apr/19 10:49
    Worklog Time Spent: 10m 
      Work Description: elek 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_r270811309
 
 

 ##########
 File path: 
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneClientAdapterImpl.java
 ##########
 @@ -0,0 +1,371 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.ozone;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
+
+import org.apache.commons.lang3.StringUtils;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Basic Implementation of the OzoneFileSystem calls.
+ * <p>
+ * This is the minimal version which doesn't include any statistics.
+ * <p>
+ * For full featured version use OzoneClientAdapterImpl.
+ */
+public class BasicOzoneClientAdapterImpl implements OzoneClientAdapter {
+
+  static final Logger LOG =
+      LoggerFactory.getLogger(BasicOzoneClientAdapterImpl.class);
+
+  private OzoneClient ozoneClient;
+  private ObjectStore objectStore;
+  private OzoneVolume volume;
+  private OzoneBucket bucket;
+  private ReplicationType replicationType;
+  private ReplicationFactor replicationFactor;
+  private boolean securityEnabled;
+
+  /**
+   * Create new OzoneClientAdapter implementation.
+   *
+   * @param volumeStr Name of the volume to use.
+   * @param bucketStr Name of the bucket to use
+   * @throws IOException In case of a problem.
+   */
+  public BasicOzoneClientAdapterImpl(String volumeStr, String bucketStr)
+      throws IOException {
+    this(createConf(), volumeStr, bucketStr);
+  }
+
+  private static OzoneConfiguration createConf() {
+    ClassLoader contextClassLoader =
+        Thread.currentThread().getContextClassLoader();
+    Thread.currentThread().setContextClassLoader(null);
+    OzoneConfiguration conf = new OzoneConfiguration();
+    Thread.currentThread().setContextClassLoader(contextClassLoader);
+    return conf;
+  }
+
+  public BasicOzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr,
+      String bucketStr)
+      throws IOException {
+    this(null, -1, conf, volumeStr, bucketStr);
+  }
+
+  public BasicOzoneClientAdapterImpl(String omHost, int omPort,
+      Configuration hadoopConf, String volumeStr, String bucketStr)
+      throws IOException {
+
+    ClassLoader contextClassLoader =
+        Thread.currentThread().getContextClassLoader();
+    Thread.currentThread().setContextClassLoader(null);
+    OzoneConfiguration conf;
+    if (hadoopConf instanceof OzoneConfiguration) {
+      conf = (OzoneConfiguration) hadoopConf;
+    } else {
+      conf = new OzoneConfiguration(hadoopConf);
+    }
+
+    SecurityConfig secConfig = new SecurityConfig(conf);
+
+    if (secConfig.isSecurityEnabled()) {
+      this.securityEnabled = true;
+    }
+
+    try {
+      String replicationTypeConf =
+          conf.get(OzoneConfigKeys.OZONE_REPLICATION_TYPE,
+              OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT);
+
+      int replicationCountConf = conf.getInt(OzoneConfigKeys.OZONE_REPLICATION,
+          OzoneConfigKeys.OZONE_REPLICATION_DEFAULT);
+
+      if (StringUtils.isNotEmpty(omHost) && omPort != -1) {
 
 Review comment:
   IMHO this part is not modified in this patch (I just renamed the class). I 
agree that It can be improved but I would prefer to do it in a separated jira 
to make it easier to follow what has been changed.
 
----------------------------------------------------------------
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: 221225)
    Time Spent: 2h 50m  (was: 2h 40m)

> 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: 2h 50m
>  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