You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by je...@apache.org on 2014/02/19 02:30:32 UTC

git commit: phoenix-42: Provide end2end test command line utility

Repository: incubator-phoenix
Updated Branches:
  refs/heads/master fcc987f9b -> 58ab608dc


phoenix-42: Provide end2end test command line utility


Project: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/commit/58ab608d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/tree/58ab608d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/diff/58ab608d

Branch: refs/heads/master
Commit: 58ab608dc79f88fa2a9ab59c6b0603c2e6168f8e
Parents: fcc987f
Author: Jeffrey Zhong <jz...@JZhongs-MacBook-Pro.local>
Authored: Tue Feb 18 17:26:55 2014 -0800
Committer: Jeffrey Zhong <jz...@JZhongs-MacBook-Pro.local>
Committed: Tue Feb 18 17:31:25 2014 -0800

----------------------------------------------------------------------
 bin/end2endTest.sh                              |  46 +++++
 .../phoenix/end2end/BaseConnectedQueryTest.java |  23 ++-
 .../ConnectionQueryServicesTestImpl.java        |  34 +++-
 .../phoenix/end2end/End2EndTestDriver.java      | 194 +++++++++++++++++++
 4 files changed, 292 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/58ab608d/bin/end2endTest.sh
----------------------------------------------------------------------
diff --git a/bin/end2endTest.sh b/bin/end2endTest.sh
new file mode 100755
index 0000000..a8beb40
--- /dev/null
+++ b/bin/end2endTest.sh
@@ -0,0 +1,46 @@
+#!/bin/bash
+#
+#/**
+# * 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.
+# */
+#
+
+current_dir=$(cd $(dirname $0);pwd)
+phoenix_jar_path="$current_dir/../phoenix-core/target/*"
+
+# check envvars which might override default args
+if [ "${PHOENIX_LIB_DIR}" != "" ]; then
+  phoenix_jar_path="${PHOENIX_LIB_DIR}"
+fi
+
+# HBase configuration folder path (where hbase-site.xml reside) for HBase/Phoenix client side property override
+hbase_config_path="$current_dir"
+# check envvars which might override default args
+if [ "${HBASE_CONF_DIR}" != "" ]; then
+  hbase_config_path="${HBASE_CONF_DIR}"
+fi
+
+hbase_library_path="$current_dir"
+# check envvars which might override default args
+if [ "${HBASE_LIBRARY_DIR}" != "" ]; then
+  #Sample class path would be: /usr/lib/hbase-0.94.15/lib/*:/usr/lib/hbase-0.94.15/*
+  hbase_library_path="${HBASE_LIBRARY_DIR}"
+fi
+
+echo "Current ClassPath="$hbase_config_path:$hbase_library_path:$phoenix_jar_path
+
+java -cp "$hbase_config_path:$phoenix_jar_path:$hbase_library_path" org.apache.phoenix.end2end.End2EndTestDriver "$@"

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/58ab608d/phoenix-core/src/test/java/org/apache/phoenix/end2end/BaseConnectedQueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/BaseConnectedQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/BaseConnectedQueryTest.java
index bbf9342..bb55530 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/end2end/BaseConnectedQueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/BaseConnectedQueryTest.java
@@ -65,10 +65,13 @@ import java.sql.ResultSet;
 import java.sql.Types;
 import java.util.Properties;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.HBaseFactoryProvider;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
@@ -100,7 +103,25 @@ public abstract class BaseConnectedQueryTest extends BaseTest {
     }
     
     protected static String getUrl() {
-        return TEST_URL;
+      Configuration conf = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+      boolean isDistributedCluster = false;
+      isDistributedCluster =
+          Boolean.parseBoolean(System.getProperty(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER,
+            "false"));
+      if (!isDistributedCluster) {
+        isDistributedCluster =
+            conf.getBoolean(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER, false);
+      }
+      // reconstruct url when running against a live cluster
+      if (isDistributedCluster) {
+        return "jdbc:phoenix:" + conf.get(HConstants.ZOOKEEPER_QUORUM, "localhost") + ":"
+            + conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT)
+            + ":"
+            + conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT)
+            + ";test=true";
+      } else {
+        return TestUtil.PHOENIX_JDBC_URL;
+      }
     }
 
     @BeforeClass

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/58ab608d/phoenix-core/src/test/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
index c776f19..912a860 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/ConnectionQueryServicesTestImpl.java
@@ -18,10 +18,15 @@
 package org.apache.phoenix.end2end;
 
 import java.sql.SQLException;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.query.ConnectionQueryServicesImpl;
@@ -37,20 +42,41 @@ import org.apache.phoenix.query.QueryServices;
  * @since 0.1
  */
 public class ConnectionQueryServicesTestImpl extends ConnectionQueryServicesImpl {
-    private HBaseTestingUtility util;
+    protected int NUM_SLAVES_BASE = 1; // number of slaves for the cluster
+    
+    private IntegrationTestingUtility util;
 
     public ConnectionQueryServicesTestImpl(QueryServices services, ConnectionInfo info) throws SQLException {
         super(services, info);
     }
 
     private Configuration setupServer(Configuration config) throws Exception {
-        util = new HBaseTestingUtility(config);
-        util.startMiniCluster();
+        util =  new IntegrationTestingUtility(config);
+        util.initializeCluster(this.NUM_SLAVES_BASE);
         return util.getConfiguration();
     }
     
+    public boolean isDistributedCluster() {
+        Configuration conf = util.getConfiguration();
+        boolean isDistributedCluster = false;
+        isDistributedCluster = Boolean.parseBoolean(System.getProperty(util.IS_DISTRIBUTED_CLUSTER, "false"));
+        if (!isDistributedCluster) {
+          isDistributedCluster = conf.getBoolean(util.IS_DISTRIBUTED_CLUSTER, false);
+        }
+        return isDistributedCluster;
+    }
+    
     private void teardownServer() throws Exception {
-        util.shutdownMiniCluster();
+        if(isDistributedCluster()){
+            // remove all hbase tables
+            HBaseAdmin admin = util.getHBaseAdmin();
+            HTableDescriptor[] tables = admin.listTables();
+            for(HTableDescriptor table : tables){
+                util.deleteTable(table.getName());
+            }
+        } else {
+            util.restoreCluster();
+        }
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/58ab608d/phoenix-core/src/test/java/org/apache/phoenix/end2end/End2EndTestDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/End2EndTestDriver.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/End2EndTestDriver.java
new file mode 100644
index 0000000..edc94cb
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/End2EndTestDriver.java
@@ -0,0 +1,194 @@
+/*
+ * Copyright 2014 The Apache Software Foundation
+ *
+ * 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.phoenix.end2end;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.hbase.ClassFinder;
+import org.apache.hadoop.hbase.ClassTestFinder;
+import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.ClassFinder.FileNameFilter;
+import org.apache.hadoop.hbase.util.AbstractHBaseTool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.internal.TextListener;
+import org.junit.runner.Description;
+import org.junit.runner.JUnitCore;
+import org.junit.runner.Result;
+import org.junit.runner.notification.Failure;
+
+/**
+ * This class drives the End2End tests suite execution against an
+ * already deployed distributed cluster.
+ */
+public class End2EndTestDriver extends AbstractHBaseTool {
+    
+    private static final String SHORT_REGEX_ARG = "r";
+    private static final String SKIP_TESTS = "n";
+    
+    private End2EndTestFilter end2endTestFilter = new End2EndTestFilter();
+    private boolean skipTests = false;
+    
+
+    public static void main(String[] args) throws Exception {
+      int ret = ToolRunner.run(new End2EndTestDriver(), args);
+      System.exit(ret);
+    }
+    
+    public static class End2EndFileNameFilter implements FileNameFilter {
+
+        @Override
+        public boolean isCandidateFile(String fileName, String absFilePath) {
+          return fileName.contains("Test");
+        }
+    };
+
+
+    public class End2EndTestFilter extends ClassTestFinder.TestClassFilter {
+      private Pattern testFilterRe = Pattern.compile(".*end2end.*");
+      public End2EndTestFilter() {
+        super();
+      }
+
+      public void setPattern(String pattern) {
+        testFilterRe = Pattern.compile(pattern);
+      }
+
+      @Override
+      public boolean isCandidateClass(Class<?> c) {
+        return testFilterRe.matcher(c.getName()).find() &&
+          // Our pattern will match the below NON-IntegrationTest. Rather than
+          // do exotic regex, just filter it out here
+          super.isCandidateClass(c);
+      }
+    }
+
+    @Override
+    protected void addOptions() {
+      addOptWithArg(SHORT_REGEX_ARG, 
+        "Java regex to use selecting tests to run: e.g. .*TestBig.*" +
+        " will select all tests that include TestBig in their name.  Default: " +
+        ".*end2end.*");
+      addOptNoArg(SKIP_TESTS, 
+          "Print list of End2End test suits without running them.");
+    }
+
+    @Override
+    protected void processOptions(CommandLine cmd) {
+      String testFilterString = cmd.getOptionValue(SHORT_REGEX_ARG, null);
+      if (testFilterString != null) {
+        end2endTestFilter.setPattern(testFilterString);
+      }
+      skipTests = cmd.hasOption(SKIP_TESTS);
+    }
+
+    /**
+     * Returns test classes annotated with @Category(IntegrationTests.class),
+     * according to the filter specific on the command line (if any).
+     */
+    private Class<?>[] findEnd2EndTestClasses()
+      throws ClassNotFoundException, LinkageError, IOException {
+      End2EndFileNameFilter nameFilter = new End2EndFileNameFilter();
+      ClassFinder classFinder = new ClassFinder(nameFilter, end2endTestFilter);
+      Set<Class<?>> classes = classFinder.findClasses("org.apache.phoenix.end2end", true);
+      return classes.toArray(new Class<?>[classes.size()]);
+    }
+
+
+    public static class End2EndTestListenter extends TextListener {
+        private final PrintStream fWriter;
+        List<String> completes;
+        public End2EndTestListenter(PrintStream writer) {
+            super(writer);
+            completes = new ArrayList<String>();
+            fWriter = writer;
+        }
+
+        @Override
+        protected void printHeader(long runTime) {
+            fWriter.println();
+            fWriter.println("=========== Test Result ===========");
+            fWriter.println("Time: " + elapsedTimeAsString(runTime));
+        }
+        
+        @Override
+        public void testStarted(Description description) {
+            fWriter.println();
+            fWriter.println("===> " + description.getDisplayName() + " starts");
+        }
+        
+        @Override
+        public void testFinished(Description description) throws Exception {
+            super.testFinished(description);
+            completes.add(description.getDisplayName());
+        }
+        
+        void printSummary(Result result){
+            Set<String> failures = new HashSet<String>();
+            for(Failure f : result.getFailures()){
+                failures.add(f.getTestHeader());
+            }
+            fWriter.println();
+            fWriter.println("==== Test Summary ====");
+            String status;
+            for(String curTest : completes){
+                status = "passed";
+                if(failures.contains(curTest)) {
+                    status = "failed";
+                }
+                fWriter.println(curTest + "   " + status + "!");
+            }
+        }
+        
+        @Override
+        public void testRunFinished(Result result) {
+            printHeader(result.getRunTime());
+            printFailures(result);
+            printSummary(result);
+            fWriter.println();
+            printFooter(result);
+        }
+    };
+
+    
+    @Override
+    protected int doWork() throws Exception {
+      //this is called from the command line, so we should set to use the distributed cluster
+      IntegrationTestingUtility.setUseDistributedCluster(conf);
+      Class<?>[] classes = findEnd2EndTestClasses();
+      System.out.println("Found " + classes.length + " end2end tests to run:");
+      for (Class<?> aClass : classes) {
+          System.out.println("  " + aClass);
+      }
+      if(skipTests) return 0;
+      
+      JUnitCore junit = new JUnitCore();
+      junit.addListener(new End2EndTestListenter(System.out));
+      Result result = junit.run(classes);
+
+      return result.wasSuccessful() ? 0 : 1;
+    }
+}