You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mw...@apache.org on 2017/03/09 18:46:36 UTC

accumulo git commit: Accumulo-4599 Simplify how accumulo runs client jars

Repository: accumulo
Updated Branches:
  refs/heads/master cbdf02e33 -> 3020c18b9


Accumulo-4599 Simplify how accumulo runs client jars

* Improved documentation and simplified methods for running accumulo client jars
* Removed ACCUMULO_XTRAJARS variable and 'accumulo jar' command
* Removed '-add' argument for executing main class using 'accumulo' command.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/3020c18b
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/3020c18b
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/3020c18b

Branch: refs/heads/master
Commit: 3020c18b92a049a94abcdfaa3dd3c1292a756202
Parents: cbdf02e
Author: Mike Walch <mw...@apache.org>
Authored: Tue Mar 7 10:29:29 2017 -0500
Committer: Mike Walch <mw...@apache.org>
Committed: Thu Mar 9 11:01:02 2017 -0500

----------------------------------------------------------------------
 assemble/bin/accumulo                           |  32 ------
 core/src/main/findbugs/exclude-filter.xml       |   5 +-
 .../java/org/apache/accumulo/core/util/Jar.java |  69 ------------
 docs/src/main/asciidoc/chapters/clients.txt     |  33 ++++--
 .../java/org/apache/accumulo/start/Main.java    |  26 +----
 .../classloader/vfs/AccumuloVFSClassLoader.java |  16 +--
 .../org/apache/accumulo/start/MainTest.java     | 112 -------------------
 .../accumulo/test/start/KeywordStartIT.java     |   2 -
 8 files changed, 27 insertions(+), 268 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/assemble/bin/accumulo
----------------------------------------------------------------------
diff --git a/assemble/bin/accumulo b/assemble/bin/accumulo
index 04b9836..2d70a43 100755
--- a/assemble/bin/accumulo
+++ b/assemble/bin/accumulo
@@ -59,45 +59,14 @@ function main() {
   export ACCUMULO_CONF_DIR="$conf"
 
   # Verify setting in accumulo-env.sh
-
   : "${JAVA_OPTS:?"variable is not set in accumulo-env.sh"}"
   : "${ACCUMULO_LOG_DIR:?"variable is not set in accumulo-env.sh"}"
   mkdir -p "${ACCUMULO_LOG_DIR}" 2>/dev/null
-  verify_env_dir "ACCUMULO_LOG_DIR" "${ACCUMULO_LOG_DIR}"
   verify_env_dir "HADOOP_PREFIX" "${HADOOP_PREFIX}"
   verify_env_dir "HADOOP_CONF_DIR" "${HADOOP_CONF_DIR}"
   verify_env_dir "ZOOKEEPER_HOME" "${ZOOKEEPER_HOME}"
   : "${MALLOC_ARENA_MAX:?"variable is not set in accumulo-env.sh"}"
 
-  # ACCUMULO_XTRAJARS is where all of the commandline -add items go into for reading by accumulo.
-  # It also holds the JAR run with the jar command and, if possible, any items in the JAR manifest's Class-Path.
-  if [[ "$cmd" = "-add" ]]; then
-    export ACCUMULO_XTRAJARS="$2"
-    shift 2
-  else
-    export ACCUMULO_XTRAJARS=""
-  fi
-  if [[ "$cmd" = "jar" && -f "$2" ]]; then
-    if [[ $2 =~ ^/ ]]; then
-      jardir="$(dirname "$2")"
-      jarfile="$2"
-    else
-      jardir="$(pwd)"
-      jarfile="${jardir}/${2}"
-    fi
-    if jar tf "$jarfile" | grep -q META-INF/MANIFEST.MF ; then
-      cp="$(unzip -p "$jarfile" META-INF/MANIFEST.MF | grep ^Class-Path: | sed 's/^Class-Path: *//')"
-      if [[ -n "$cp" ]] ; then
-         for j in $cp; do
-            if [[ "$j" != "Class-Path:" ]] ; then
-               ACCUMULO_XTRAJARS="${jardir}/${j},$ACCUMULO_XTRAJARS"
-            fi
-         done
-      fi
-    fi
-    ACCUMULO_XTRAJARS="${jarfile},$ACCUMULO_XTRAJARS"
-  fi
-
   if [ -x "$JAVA_HOME/bin/java" ]; then
     JAVA="$JAVA_HOME/bin/java"
   else
@@ -107,7 +76,6 @@ function main() {
     echo "Could not find any executable java binary. Please set java on your PATH or set JAVA_HOME"
     exit 1
   fi
-
   if [ -n "$ACCUMULO_JAVA_PREFIX" ]; then
     JAVA=($ACCUMULO_JAVA_PREFIX $JAVA)
   fi

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/core/src/main/findbugs/exclude-filter.xml
----------------------------------------------------------------------
diff --git a/core/src/main/findbugs/exclude-filter.xml b/core/src/main/findbugs/exclude-filter.xml
index 58d7a12..95aca41 100644
--- a/core/src/main/findbugs/exclude-filter.xml
+++ b/core/src/main/findbugs/exclude-filter.xml
@@ -94,10 +94,7 @@
   </Match>
   <Match>
     <!-- admin classes can call System.exit -->
-    <Or>
-      <Class name="org.apache.accumulo.core.util.Jar" />
-      <Class name="org.apache.accumulo.core.file.rfile.PrintInfo" />
-    </Or>
+    <Class name="org.apache.accumulo.core.file.rfile.PrintInfo" />
     <Bug code="DM" pattern="DM_EXIT" />
   </Match>
   <Match>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/core/src/main/java/org/apache/accumulo/core/util/Jar.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/Jar.java b/core/src/main/java/org/apache/accumulo/core/util/Jar.java
deleted file mode 100644
index a64e444..0000000
--- a/core/src/main/java/org/apache/accumulo/core/util/Jar.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * 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.accumulo.core.util;
-
-import java.io.IOException;
-import java.util.jar.JarFile;
-
-import org.apache.accumulo.start.Main;
-import org.apache.accumulo.start.spi.KeywordExecutable;
-
-import com.google.auto.service.AutoService;
-
-@AutoService(KeywordExecutable.class)
-public class Jar implements KeywordExecutable {
-
-  @Override
-  public String keyword() {
-    return "jar";
-  }
-
-  public String usage() {
-    return "jar <jar> [<main class>] args";
-  }
-
-  @Override
-  public String description() {
-    return "Runs Java <main class> in <jar> using Accumulo classpath";
-  }
-
-  @Override
-  public void execute(final String[] args) throws Exception {
-    // need at least one argument for the jar file, two arguments if the jar file manifest doesn't specify a main class
-    if (args.length == 0) {
-      Main.printUsage();
-      System.exit(1);
-    }
-    String jarFileName = args[0];
-    String candidateMainClass = args.length > 1 ? args[1] : null;
-    Class<?> mainClass = null;
-    try {
-      JarFile f = new JarFile(jarFileName);
-      mainClass = Main.loadClassFromJar(args, f, Main.getClassLoader());
-    } catch (IOException ioe) {
-      System.out.println("File " + jarFileName + " could not be found or read.");
-      System.exit(1);
-    } catch (ClassNotFoundException cnfe) {
-      System.out.println("Classname " + (candidateMainClass != null ? candidateMainClass : "in JAR manifest")
-          + " not found.  Please make sure you use the wholly qualified package name.");
-      System.exit(1);
-    }
-    // strip the jar file name and, if specified, the main class name from the args; then execute
-    String[] newArgs = Main.stripArgs(args, mainClass.getName().equals(candidateMainClass) ? 2 : 1);
-    Main.execMainClass(mainClass, newArgs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/docs/src/main/asciidoc/chapters/clients.txt
----------------------------------------------------------------------
diff --git a/docs/src/main/asciidoc/chapters/clients.txt b/docs/src/main/asciidoc/chapters/clients.txt
index c740e6b..6e0909b 100644
--- a/docs/src/main/asciidoc/chapters/clients.txt
+++ b/docs/src/main/asciidoc/chapters/clients.txt
@@ -20,11 +20,13 @@
 There are multiple ways to run Java code that uses Accumulo. Below is a list
 of the different ways to execute client code.
 
-* using java executable
-* using the accumulo script
-* using the tool script
+* using the +java+ command
+* using the +accumulo+ command
+* using the +accumulo-util hadoop-jar+ command
 
-In order to run client code written to run against Accumulo, you will need to
+==== Using the java command
+
+To run Accumulo client code using the +java+ command, you will need to
 include the jars that Accumulo depends on in your classpath. Accumulo client
 code depends on Hadoop and Zookeeper. For Hadoop add the hadoop client jar, all
 of the jars in the Hadoop lib directory, and the conf directory to the
@@ -34,13 +36,24 @@ configured Accumulo system to see what its using for its classpath.
 
   accumulo classpath
 
-Another option for running your code is to put a jar file in
-+lib/ext+. After doing this you can use the accumulo
-script to execute your code. For example if you create a jar containing the
-class +com.foo.Client+ and placed that in +lib/ext+, then you could use the command
-+accumulo com.foo.Client+ to execute your code.
+==== Using the accumulo command
+
+Another option for running your code is to use the Accumulo script which can execute a
+main class (if it exists on its classpath):
+
+  accumulo com.foo.Client arg1 arg2
+
+While the Accumulo script will add all of Accumulo's dependencies to the classpath, you
+will need to add any jars that your create or depend on beyond what Accumulo already
+depends on. This can be accomplished by either adding the jars to the +lib/ext+ directory
+of your Accumulo installation or by adding jars to the CLASSPATH variable before calling
+the accumulo command.
+
+  export CLASSPATH=/path/to/my.jar:/path/to/dep.jar; accumulo com.foo.Client arg1 arg2
+
+==== Using the 'accumulo-util hadoop-jar' command
 
-If you are writing map reduce job that access Accumulo, then you can use
+If you are writing map reduce job that accesses Accumulo, then you can use
 +accumulo-util hadoop-jar+ to run those jobs. See the map reduce example.
 
 === Connecting

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/start/src/main/java/org/apache/accumulo/start/Main.java
----------------------------------------------------------------------
diff --git a/start/src/main/java/org/apache/accumulo/start/Main.java b/start/src/main/java/org/apache/accumulo/start/Main.java
index c0e8c2b..ea2e1b8 100644
--- a/start/src/main/java/org/apache/accumulo/start/Main.java
+++ b/start/src/main/java/org/apache/accumulo/start/Main.java
@@ -26,8 +26,6 @@ import java.util.Map;
 import java.util.ServiceLoader;
 import java.util.TreeMap;
 import java.util.TreeSet;
-import java.util.jar.Attributes;
-import java.util.jar.JarFile;
 
 import org.apache.accumulo.start.classloader.AccumuloClassLoader;
 import org.apache.accumulo.start.spi.KeywordExecutable;
@@ -208,7 +206,7 @@ public class Main {
     Map<String,KeywordExecutable> executableMap = new TreeMap<>(getExecutables(getClassLoader()));
 
     System.out.println("\nUsage: accumulo <command> (<argument> ...)\n\nCore Commands:");
-    for (String cmd : Arrays.asList("init", "shell", "classpath", "version", "admin", "info", "help", "jar")) {
+    for (String cmd : Arrays.asList("init", "shell", "classpath", "version", "admin", "info", "help")) {
       printCommand(executableMap.remove(cmd));
     }
     System.out.println("  <main class> args              Runs Java <main class> located on Accumulo classpath");
@@ -254,26 +252,4 @@ public class Main {
   private static void warnDuplicate(final KeywordExecutable service) {
     log.warn("Ambiguous duplicate binding for keyword '" + service.keyword() + "' found: " + service.getClass().getName());
   }
-
-  // feature: will work even if main class isn't in the JAR
-  public static Class<?> loadClassFromJar(final String[] args, final JarFile f, final ClassLoader cl) throws ClassNotFoundException, IOException {
-    ClassNotFoundException explicitNotFound = null;
-    if (args.length >= 2) {
-      try {
-        return cl.loadClass(args[1]); // jar-file main-class
-      } catch (ClassNotFoundException cnfe) {
-        // assume this is the first argument, look for main class in JAR manifest
-        explicitNotFound = cnfe;
-      }
-    }
-    String mainClass = f.getManifest().getMainAttributes().getValue(Attributes.Name.MAIN_CLASS);
-    if (mainClass == null) {
-      if (explicitNotFound != null) {
-        throw explicitNotFound;
-      }
-      throw new ClassNotFoundException("No main class was specified, and the JAR manifest does not specify one");
-    }
-    return cl.loadClass(mainClass);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
----------------------------------------------------------------------
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
index f287364..8e6a84b 100644
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
+++ b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/AccumuloVFSClassLoader.java
@@ -171,22 +171,10 @@ public class AccumuloVFSClassLoader {
     return classpath.toArray(new FileObject[classpath.size()]);
   }
 
-  private static ReloadingClassLoader createDynamicClassloader(final ClassLoader parent) throws FileSystemException, IOException {
+  private static ReloadingClassLoader createDynamicClassloader(final ClassLoader parent) throws IOException {
     String dynamicCPath = AccumuloClassLoader.getAccumuloString(DYNAMIC_CLASSPATH_PROPERTY_NAME, DEFAULT_DYNAMIC_CLASSPATH_VALUE);
 
-    String envJars = System.getenv("ACCUMULO_XTRAJARS");
-    if (null != envJars && !envJars.equals(""))
-      if (dynamicCPath != null && !dynamicCPath.equals(""))
-        dynamicCPath = dynamicCPath + "," + envJars;
-      else
-        dynamicCPath = envJars;
-
-    ReloadingClassLoader wrapper = new ReloadingClassLoader() {
-      @Override
-      public ClassLoader getClassLoader() {
-        return parent;
-      }
-    };
+    ReloadingClassLoader wrapper = () -> parent;
 
     if (dynamicCPath == null || dynamicCPath.equals(""))
       return wrapper;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/start/src/test/java/org/apache/accumulo/start/MainTest.java
----------------------------------------------------------------------
diff --git a/start/src/test/java/org/apache/accumulo/start/MainTest.java b/start/src/test/java/org/apache/accumulo/start/MainTest.java
deleted file mode 100644
index 0e6102d..0000000
--- a/start/src/test/java/org/apache/accumulo/start/MainTest.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * 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.accumulo.start;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertEquals;
-
-import java.util.jar.Attributes;
-import java.util.jar.JarFile;
-import java.util.jar.Manifest;
-
-import org.easymock.EasyMock;
-import org.junit.Before;
-import org.junit.Test;
-
-public class MainTest {
-  private static final Class<?> MAIN_CLASS = String.class; // arbitrary
-  private static final String MAIN_CLASS_NAME = MAIN_CLASS.getName();
-
-  private JarFile f;
-  private ClassLoader cl;
-
-  @Before
-  public void setUp() {
-    f = createMock(JarFile.class);
-    cl = createMock(ClassLoader.class);
-  }
-
-  @Test
-  public void testLoadClassFromJar_ExplicitMainClass() throws Exception {
-    String[] args = {"the.jar", "main.class", "arg1", "arg2"};
-    EasyMock.<Class<?>> expect(cl.loadClass("main.class")).andReturn(MAIN_CLASS);
-    replay(cl);
-    assertEquals(MAIN_CLASS, Main.loadClassFromJar(args, f, cl));
-  }
-
-  @Test
-  public void testLoadClassFromJar_ManifestMainClass() throws Exception {
-    String[] args = {"the.jar", "arg1", "arg2"};
-    expect(cl.loadClass("arg1")).andThrow(new ClassNotFoundException());
-    EasyMock.<Class<?>> expect(cl.loadClass(MAIN_CLASS_NAME)).andReturn(MAIN_CLASS);
-    replay(cl);
-    mockManifestMainClass(f, MAIN_CLASS.getName());
-    replay(f);
-    assertEquals(MAIN_CLASS, Main.loadClassFromJar(args, f, cl));
-  }
-
-  @Test(expected = ClassNotFoundException.class)
-  public void testLoadClassFromJar_NoMainClass() throws Exception {
-    String[] args = {"the.jar", "arg1", "arg2"};
-    expect(cl.loadClass("arg1")).andThrow(new ClassNotFoundException());
-    replay(cl);
-    mockManifestMainClass(f, null);
-    replay(f);
-    Main.loadClassFromJar(args, f, cl);
-  }
-
-  @Test(expected = ClassNotFoundException.class)
-  public void testLoadClassFromJar_NoMainClassNoArgs() throws Exception {
-    String[] args = {"the.jar"};
-    mockManifestMainClass(f, null);
-    replay(f);
-    Main.loadClassFromJar(args, f, cl);
-  }
-
-  @Test(expected = ClassNotFoundException.class)
-  public void testLoadClassFromJar_ExplicitMainClass_Fail() throws Exception {
-    String[] args = {"the.jar", "main.class", "arg1", "arg2"};
-    expect(cl.loadClass("main.class")).andThrow(new ClassNotFoundException());
-    replay(cl);
-    mockManifestMainClass(f, null);
-    replay(f);
-    Main.loadClassFromJar(args, f, cl);
-  }
-
-  @Test(expected = ClassNotFoundException.class)
-  public void testLoadClassFromJar_ManifestMainClass_Fail() throws Exception {
-    String[] args = {"the.jar", "arg1", "arg2"};
-    expect(cl.loadClass("arg1")).andThrow(new ClassNotFoundException());
-    expect(cl.loadClass(MAIN_CLASS_NAME)).andThrow(new ClassNotFoundException());
-    replay(cl);
-    mockManifestMainClass(f, MAIN_CLASS.getName());
-    replay(f);
-    Main.loadClassFromJar(args, f, cl);
-  }
-
-  private void mockManifestMainClass(JarFile f, String mainClassName) throws Exception {
-    Manifest mf = createMock(Manifest.class);
-    expect(f.getManifest()).andReturn(mf);
-    Attributes attrs = createMock(Attributes.class);
-    expect(mf.getMainAttributes()).andReturn(attrs);
-    replay(mf);
-    expect(attrs.getValue(Attributes.Name.MAIN_CLASS)).andReturn(mainClassName);
-    replay(attrs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3020c18b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
index b32382c..3b67e1f 100644
--- a/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/start/KeywordStartIT.java
@@ -37,7 +37,6 @@ import org.apache.accumulo.core.file.rfile.PrintInfo;
 import org.apache.accumulo.core.util.Classpath;
 import org.apache.accumulo.core.util.CreateToken;
 import org.apache.accumulo.core.util.Help;
-import org.apache.accumulo.core.util.Jar;
 import org.apache.accumulo.core.util.Version;
 import org.apache.accumulo.gc.GCExecutable;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
@@ -108,7 +107,6 @@ public class KeywordStartIT {
     expectSet.put("help", Help.class);
     expectSet.put("info", Info.class);
     expectSet.put("init", Initialize.class);
-    expectSet.put("jar", Jar.class);
     expectSet.put("login-info", LoginProperties.class);
     expectSet.put("master", MasterExecutable.class);
     expectSet.put("minicluster", MiniClusterExecutable.class);