You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/07/20 19:14:47 UTC

[17/50] [abbrv] phoenix git commit: PHOENIX-2103 : Pig tests aren't dropping tables as expected between test runs

PHOENIX-2103 : Pig tests aren't dropping tables as expected between test runs


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

Branch: refs/heads/calcite
Commit: 984e62223c8aa507e9c044cecfc7fc92ffa42522
Parents: 11bdb0e
Author: ravimagham <ra...@apache.org>
Authored: Thu Jul 9 20:20:32 2015 -0700
Committer: ravimagham <ra...@apache.org>
Committed: Thu Jul 9 20:20:32 2015 -0700

----------------------------------------------------------------------
 .../phoenix/pig/PhoenixHBaseLoaderIT.java       | 134 ++++++++-----------
 1 file changed, 53 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/984e6222/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
index 7fcf6ac..53a62ee 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
@@ -19,9 +19,11 @@
  */
 package org.apache.phoenix.pig;
 
-import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -35,18 +37,11 @@ import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest;
-import org.apache.phoenix.jdbc.PhoenixDriver;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.TestUtil;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.DataType;
@@ -54,20 +49,15 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.google.common.base.Preconditions;
 
 /**
  * 
  * Test class to run all the integration tests against a virtual map reduce cluster.
  */
-@Category(NeedsOwnMiniClusterTest.class)
-public class PhoenixHBaseLoaderIT {
+public class PhoenixHBaseLoaderIT extends BaseHBaseManagedTimeIT {
     
     private static final Log LOG = LogFactory.getLog(PhoenixHBaseLoaderIT.class);
     private static final String SCHEMA_NAME = "T";
@@ -76,32 +66,16 @@ public class PhoenixHBaseLoaderIT {
     private static final String TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, TABLE_NAME);
     private static final String CASE_SENSITIVE_TABLE_NAME = SchemaUtil.getEscapedArgument("a");
     private static final String CASE_SENSITIVE_TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME,CASE_SENSITIVE_TABLE_NAME);
-    private static HBaseTestingUtility hbaseTestUtil;
-    private static String zkQuorum;
-    private static Connection conn;
-    private static PigServer pigServer;
-    private static Configuration conf;
-
-    @BeforeClass
-    public static void setUpBeforeClass() throws Exception {
-        hbaseTestUtil = new HBaseTestingUtility();
-        conf = hbaseTestUtil.getConfiguration();
-        setUpConfigForMiniCluster(conf);
-        conf.set(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        hbaseTestUtil.startMiniCluster();
+    private String zkQuorum;
+    private Connection conn;
+    private PigServer pigServer;
 
-        Class.forName(PhoenixDriver.class.getName());
-        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
-        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
-        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
-        conn = DriverManager.getConnection(PhoenixRuntime.JDBC_PROTOCOL +
-                 PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum,props);
-     }
-    
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL,
-                ConfigurationUtil.toProperties(conf));
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        conn = DriverManager.getConnection(getUrl(), props);
+        zkQuorum = LOCALHOST + JDBC_PROTOCOL_SEPARATOR + getZKClientPort(getTestClusterConfig());
+        pigServer = new PigServer(ExecType.LOCAL, getTestClusterConfig());
     }
 
     /**
@@ -110,13 +84,14 @@ public class PhoenixHBaseLoaderIT {
      */
     @Test
     public void testSchemaForTable() throws Exception {
+        final String TABLE = "TABLE1";
         final String ddl = String.format("CREATE TABLE %s "
                 + "  (a_string varchar not null, a_binary varbinary not null, a_integer integer, cf1.a_float float"
-                + "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n", TABLE_FULL_NAME);
+                + "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n", TABLE);
         conn.createStatement().execute(ddl);
-
+        conn.commit();
         pigServer.registerQuery(String.format(
-                "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE_FULL_NAME,
+                "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE,
                 zkQuorum));
         
         final Schema schema = pigServer.dumpSchema("A");
@@ -140,15 +115,15 @@ public class PhoenixHBaseLoaderIT {
     public void testSchemaForTableWithSpecificColumns() throws Exception {
         
         //create the table
-        final String ddl = "CREATE TABLE " + TABLE_FULL_NAME 
+        final String TABLE = "TABLE2";
+        final String ddl = "CREATE TABLE " + TABLE
                 + "  (ID INTEGER NOT NULL PRIMARY KEY,NAME VARCHAR, AGE INTEGER) ";
         conn.createStatement().execute(ddl);
         
-        
         final String selectColumns = "ID,NAME";
         pigServer.registerQuery(String.format(
                 "A = load 'hbase://table/%s/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');",
-                TABLE_FULL_NAME, selectColumns, zkQuorum));
+                TABLE, selectColumns, zkQuorum));
         
         Schema schema = pigServer.dumpSchema("A");
         List<FieldSchema> fields = schema.getFields();
@@ -157,7 +132,6 @@ public class PhoenixHBaseLoaderIT {
         assertTrue(fields.get(0).type == DataType.INTEGER);
         assertTrue(fields.get(1).alias.equalsIgnoreCase("NAME"));
         assertTrue(fields.get(1).type == DataType.CHARARRAY);
-        
     }
     
     /**
@@ -168,15 +142,16 @@ public class PhoenixHBaseLoaderIT {
     public void testSchemaForQuery() throws Exception {
         
        //create the table.
-        String ddl = String.format("CREATE TABLE " + TABLE_FULL_NAME +
+        final String TABLE = "TABLE3";
+        String ddl = String.format("CREATE TABLE " + TABLE +
                  "  (A_STRING VARCHAR NOT NULL, A_DECIMAL DECIMAL NOT NULL, CF1.A_INTEGER INTEGER, CF2.A_DOUBLE DOUBLE"
-                + "  CONSTRAINT pk PRIMARY KEY (A_STRING, A_DECIMAL))\n", TABLE_FULL_NAME);
+                + "  CONSTRAINT pk PRIMARY KEY (A_STRING, A_DECIMAL))\n", TABLE);
         conn.createStatement().execute(ddl);
-        
+
         
         
         //sql query for LOAD
-        final String sqlQuery = "SELECT A_STRING,CF1.A_INTEGER,CF2.A_DOUBLE FROM " + TABLE_FULL_NAME;
+        final String sqlQuery = "SELECT A_STRING,CF1.A_INTEGER,CF2.A_DOUBLE FROM " + TABLE;
         pigServer.registerQuery(String.format(
                 "A = load 'hbase://query/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');",
                 sqlQuery, zkQuorum));
@@ -201,13 +176,14 @@ public class PhoenixHBaseLoaderIT {
     public void testSchemaForTableWithAlias() throws Exception {
         
         //create the table.
-        String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+        final String TABLE = "S.TABLE4";
+        String ddl = "CREATE TABLE  " + TABLE
                 + "  (A_STRING VARCHAR NOT NULL, A_DECIMAL DECIMAL NOT NULL, CF1.A_INTEGER INTEGER, CF2.A_DOUBLE DOUBLE"
                 + "  CONSTRAINT pk PRIMARY KEY (A_STRING, A_DECIMAL)) \n";
         conn.createStatement().execute(ddl);
-        
+
         //select query given as part of LOAD.
-        final String sqlQuery = "SELECT A_STRING,A_DECIMAL,CF1.A_INTEGER,CF2.A_DOUBLE FROM " + TABLE_FULL_NAME;
+        final String sqlQuery = "SELECT A_STRING,A_DECIMAL,CF1.A_INTEGER,CF2.A_DOUBLE FROM " + TABLE;
         
         LOG.info(String.format("Generated SQL Query [%s]",sqlQuery));
         
@@ -317,13 +293,14 @@ public class PhoenixHBaseLoaderIT {
     public void testForNonPKSQLQuery() throws Exception {
         
          //create the table
-         String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+        final String TABLE = "TABLE5";
+        String ddl = "CREATE TABLE  " + TABLE
                 + " ( ID VARCHAR PRIMARY KEY, FOO VARCHAR, BAR INTEGER, BAZ UNSIGNED_INT)";
                 
         conn.createStatement().execute(ddl);
         
         //upsert data.
-        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?,?) ";
+        final String dml = "UPSERT INTO " + TABLE + " VALUES(?,?,?,?) ";
         PreparedStatement stmt = conn.prepareStatement(dml);
         stmt.setString(1, "a");
         stmt.setString(2, "a");
@@ -339,7 +316,7 @@ public class PhoenixHBaseLoaderIT {
         conn.commit();
         
         //sql query
-        final String sqlQuery = String.format(" SELECT FOO, BAZ FROM %s WHERE BAR = -1 " , TABLE_FULL_NAME);
+        final String sqlQuery = String.format(" SELECT FOO, BAZ FROM %s WHERE BAR = -1 " , TABLE);
       
         pigServer.registerQuery(String.format(
                 "A = load 'hbase://query/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", sqlQuery,
@@ -372,13 +349,14 @@ public class PhoenixHBaseLoaderIT {
     public void testGroupingOfDataForTable() throws Exception {
         
          //create the table
-         String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+        final String TABLE = "TABLE6";
+        String ddl = "CREATE TABLE  " + TABLE
                 + "  (ID  INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER, SAL INTEGER) ";
                 
         conn.createStatement().execute(ddl);
         
         //prepare data with 10 rows having age 25 and the other 30.
-        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?,?)";
+        final String dml = "UPSERT INTO " + TABLE + " VALUES(?,?,?,?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         int rows = 20;
         int j = 0, k = 0;
@@ -406,7 +384,7 @@ public class PhoenixHBaseLoaderIT {
          //load data and filter rows whose age is > 25
         pigServer.setBatchOn();
         pigServer.registerQuery(String.format(
-                "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE_FULL_NAME,
+                "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE,
                 zkQuorum));
         
         pigServer.registerQuery("B = GROUP A BY AGE;");
@@ -426,18 +404,19 @@ public class PhoenixHBaseLoaderIT {
     public void testLoadAndStore() throws Exception {
         
          //create the tables
-         final String sourceTableddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+        final String TABLE = "TABLE7";
+        final String sourceTableddl = "CREATE TABLE  " + TABLE
                 + "  (ID  INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER, SAL INTEGER) ";
          
-         final String targetTable = "AGGREGATE";
-         final String targetTableddl = "CREATE TABLE " + targetTable 
+        final String targetTable = "AGGREGATE";
+        final String targetTableddl = "CREATE TABLE " + targetTable
                  +  "(AGE INTEGER NOT NULL PRIMARY KEY , MIN_SAL INTEGER , MAX_SAL INTEGER) ";
                  
         conn.createStatement().execute(sourceTableddl);
         conn.createStatement().execute(targetTableddl);
         
         //prepare data with 10 rows having age 25 and the other 30.
-        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?,?)";
+        final String dml = "UPSERT INTO " + TABLE + " VALUES(?,?,?,?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         int rows = 20;
         int j = 0, k = 0;
@@ -460,7 +439,7 @@ public class PhoenixHBaseLoaderIT {
          //load data and filter rows whose age is > 25
         pigServer.setBatchOn();
         pigServer.registerQuery(String.format(
-                "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE_FULL_NAME,
+                "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE,
                 zkQuorum));
         
         pigServer.registerQuery("B = GROUP A BY AGE;");
@@ -491,7 +470,8 @@ public class PhoenixHBaseLoaderIT {
     public void testDataForSQLQueryWithSequences() throws Exception {
         
          //create the table
-         String ddl = "CREATE TABLE " + TABLE_FULL_NAME
+        final String TABLE = "TABLE8";
+        String ddl = "CREATE TABLE " + TABLE
                 + " (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER) ";
                 
         conn.createStatement().execute(ddl);
@@ -501,7 +481,7 @@ public class PhoenixHBaseLoaderIT {
         conn.createStatement().execute(sequenceDdl);
            
         //prepare data with 10 rows having age 25 and the other 30.
-        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?)";
+        final String dml = "UPSERT INTO " + TABLE + " VALUES(?,?,?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         int rows = 20;
         for(int i = 0 ; i < rows; i++) {
@@ -513,7 +493,7 @@ public class PhoenixHBaseLoaderIT {
         conn.commit();
         
         //sql query load data and filter rows whose age is > 25
-        final String sqlQuery = " SELECT NEXT VALUE FOR my_sequence AS my_seq,ID,NAME,AGE FROM " + TABLE_FULL_NAME + " WHERE AGE > 25";
+        final String sqlQuery = " SELECT NEXT VALUE FOR my_sequence AS my_seq,ID,NAME,AGE FROM " + TABLE + " WHERE AGE > 25";
         pigServer.registerQuery(String.format(
                 "A = load 'hbase://query/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", sqlQuery,
                 zkQuorum));
@@ -532,12 +512,13 @@ public class PhoenixHBaseLoaderIT {
     public void testDataForSQLQueryWithFunctions() throws Exception {
         
          //create the table
-         String ddl = "CREATE TABLE " + TABLE_FULL_NAME
+         final String TABLE = "TABLE9";
+         String ddl = "CREATE TABLE " + TABLE
                 + " (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR) ";
                 
         conn.createStatement().execute(ddl);
         
-        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?)";
+        final String dml = "UPSERT INTO " + TABLE + " VALUES(?,?)";
         PreparedStatement stmt = conn.prepareStatement(dml);
         int rows = 20;
         for(int i = 0 ; i < rows; i++) {
@@ -548,7 +529,7 @@ public class PhoenixHBaseLoaderIT {
         conn.commit();
         
         //sql query
-        final String sqlQuery = " SELECT UPPER(NAME) AS n FROM " + TABLE_FULL_NAME + " ORDER BY ID" ;
+        final String sqlQuery = " SELECT UPPER(NAME) AS n FROM " + TABLE + " ORDER BY ID" ;
 
         pigServer.registerQuery(String.format(
                 "A = load 'hbase://query/%s' using "  + PhoenixHBaseLoader.class.getName() + "('%s');", sqlQuery,
@@ -604,23 +585,14 @@ public class PhoenixHBaseLoaderIT {
     
     @After
     public void tearDown() throws Exception {
-        dropTable(TABLE_FULL_NAME);
-        dropTable(CASE_SENSITIVE_TABLE_FULL_NAME);
+        if(conn != null) {
+            conn.close();
+        }
         pigServer.shutdown();
     }
 
-
     private void dropTable(String tableFullName) throws SQLException {
       Preconditions.checkNotNull(conn);
       conn.createStatement().execute(String.format("DROP TABLE IF EXISTS %s",tableFullName));
     }
-
-    @AfterClass
-    public static void tearDownAfterClass() throws Exception {
-        try {
-            conn.close();
-        } finally {
-            hbaseTestUtil.shutdownMiniCluster();
-        }
-    }
 }
\ No newline at end of file