You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kh...@apache.org on 2014/10/01 11:44:02 UTC

svn commit: r1628660 - in /hive/trunk: hcatalog/hcatalog-pig-adapter/ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/ ql/src/test/org/apache/hadoop/hive/ql/io/

Author: khorgath
Date: Wed Oct  1 09:44:02 2014
New Revision: 1628660

URL: http://svn.apache.org/r1628660
Log:
HIVE-7420 : Parameterize tests for HCatalog Pig interfaces for testing against all storage formats (David Chen via Sushanth Sowmyan)

Added:
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestUtil.java
Removed:
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestOrcHCatLoader.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestOrcHCatLoaderComplexSchema.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestOrcHCatStorer.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestOrcHCatStorerMulti.java
Modified:
    hive/trunk/hcatalog/hcatalog-pig-adapter/pom.xml
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorer.java
    hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/pom.xml?rev=1628660&r1=1628659&r2=1628660&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/pom.xml (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/pom.xml Wed Oct  1 09:44:02 2014
@@ -53,6 +53,13 @@
       <classifier>tests</classifier>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java?rev=1628660&r1=1628659&r2=1628660&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoader.java Wed Oct  1 09:44:02 2014
@@ -28,10 +28,12 @@ import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 
@@ -42,6 +44,8 @@ import org.apache.hadoop.hive.cli.CliSes
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
@@ -69,12 +73,16 @@ import org.joda.time.DateTime;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
+@RunWith(Parameterized.class)
 public class TestHCatLoader {
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatLoader.class);
   private static final String TEST_DATA_DIR = HCatUtil.makePathASafeFileName(System.getProperty("java.io.tmpdir") +
@@ -91,9 +99,30 @@ public class TestHCatLoader {
   private Driver driver;
   private Map<Integer, Pair<Integer, String>> basicInputData;
 
-  protected String storageFormat() {
-    return "RCFILE tblproperties('hcat.isd'='org.apache.hive.hcatalog.rcfile.RCFileInputDriver'," +
-      "'hcat.osd'='org.apache.hive.hcatalog.rcfile.RCFileOutputDriver')";
+  private static final Map<String, Set<String>> DISABLED_STORAGE_FORMATS =
+      new HashMap<String, Set<String>>() {{
+        put(IOConstants.AVRO, new HashSet<String>() {{
+          add("testReadDataBasic");
+          add("testReadPartitionedBasic");
+          add("testProjectionsBasic");
+          add("testSchemaLoadPrimitiveTypes");
+        }});
+        put(IOConstants.PARQUETFILE, new HashSet<String>() {{
+          add("testReadDataBasic");
+          add("testReadPartitionedBasic");
+          add("testProjectionsBasic");
+        }});
+      }};
+
+  private String storageFormat;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return StorageFormats.names();
+  }
+
+  public TestHCatLoader(String storageFormat) {
+    this.storageFormat = storageFormat;
   }
 
   private void dropTable(String tablename) throws IOException, CommandNeedRetryException {
@@ -105,7 +134,7 @@ public class TestHCatLoader {
   }
 
   private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException {
-    createTable(tablename, schema, partitionedBy, driver, storageFormat());
+    createTable(tablename, schema, partitionedBy, driver, storageFormat);
   }
 
   static void createTable(String tablename, String schema, String partitionedBy, Driver driver, String storageFormat)
@@ -209,17 +238,18 @@ public class TestHCatLoader {
     server.registerQuery("D = load '" + COMPLEX_FILE_NAME + "' as (name:chararray, studentid:int, contact:tuple(phno:chararray,email:chararray), currently_registered_courses:bag{innertup:tuple(course:chararray)}, current_grades:map[ ] , phnos :bag{innertup:tuple(phno:chararray,type:chararray)});", ++i);
     server.registerQuery("store D into '" + COMPLEX_TABLE + "' using org.apache.hive.hcatalog.pig.HCatStorer();", ++i);
     server.executeBatch();
-
   }
 
   @After
   public void tearDown() throws Exception {
     try {
-      dropTable(BASIC_TABLE);
-      dropTable(COMPLEX_TABLE);
-      dropTable(PARTITIONED_TABLE);
-      dropTable(SPECIFIC_SIZE_TABLE);
-      dropTable(AllTypesTable.ALL_PRIMITIVE_TYPES_TABLE);
+      if (driver != null) {
+        dropTable(BASIC_TABLE);
+        dropTable(COMPLEX_TABLE);
+        dropTable(PARTITIONED_TABLE);
+        dropTable(SPECIFIC_SIZE_TABLE);
+        dropTable(AllTypesTable.ALL_PRIMITIVE_TYPES_TABLE);
+      }
     } finally {
       FileUtils.deleteDirectory(new File(TEST_DATA_DIR));
     }
@@ -227,6 +257,7 @@ public class TestHCatLoader {
 
   @Test
   public void testSchemaLoadBasic() throws IOException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     PigServer server = new PigServer(ExecType.LOCAL);
 
@@ -241,23 +272,28 @@ public class TestHCatLoader {
     assertTrue(Xfields.get(1).type == DataType.CHARARRAY);
 
   }
+
   /**
    * Test that we properly translate data types in Hive/HCat table schema into Pig schema
    */
   @Test
   public void testSchemaLoadPrimitiveTypes() throws IOException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     AllTypesTable.testSchemaLoadPrimitiveTypes();
   }
+
   /**
    * Test that value from Hive table are read properly in Pig
    */
   @Test
   public void testReadDataPrimitiveTypes() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     AllTypesTable.testReadDataPrimitiveTypes();
   }
 
   @Test
   public void testReadDataBasic() throws IOException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     PigServer server = new PigServer(ExecType.LOCAL);
 
     server.registerQuery("X = load '" + BASIC_TABLE + "' using org.apache.hive.hcatalog.pig.HCatLoader();");
@@ -279,6 +315,7 @@ public class TestHCatLoader {
 
   @Test
   public void testSchemaLoadComplex() throws IOException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     PigServer server = new PigServer(ExecType.LOCAL);
 
@@ -337,6 +374,7 @@ public class TestHCatLoader {
 
   @Test
   public void testReadPartitionedBasic() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     PigServer server = new PigServer(ExecType.LOCAL);
 
     driver.run("select * from " + PARTITIONED_TABLE);
@@ -404,6 +442,7 @@ public class TestHCatLoader {
 
   @Test
   public void testProjectionsBasic() throws IOException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     PigServer server = new PigServer(ExecType.LOCAL);
 
@@ -453,6 +492,7 @@ public class TestHCatLoader {
 
   @Test
   public void testColumnarStorePushdown() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     String PIGOUTPUT_DIR = TEST_DATA_DIR+ "/colpushdownop";
     String PIG_FILE = "test.pig";
     String expectedCols = "0,1";
@@ -486,6 +526,7 @@ public class TestHCatLoader {
 
   @Test
   public void testGetInputBytes() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     File file = new File(TEST_WAREHOUSE_DIR + "/" + SPECIFIC_SIZE_TABLE + "/part-m-00000");
     file.deleteOnExit();
     RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
@@ -501,6 +542,7 @@ public class TestHCatLoader {
 
   @Test
   public void testConvertBooleanToInt() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     String tbl = "test_convert_boolean_to_int";
     String inputFileName = TEST_DATA_DIR + "/testConvertBooleanToInt/data.txt";
     File inputDataDir = new File(inputFileName).getParentFile();
@@ -608,22 +650,26 @@ public class TestHCatLoader {
         Tuple t = XIter.next();
         assertEquals(HCatFieldSchema.Type.numPrimitiveTypes(), t.size());
         int colPos = 0;
-        for(Object referenceData : primitiveRows[numTuplesRead]) {
-          if(referenceData == null) {
-            assertTrue("rowNum=" + numTuplesRead + " colNum=" + colPos + " Reference data is null; actual " +
-                t.get(colPos), t.get(colPos) == null);
-          }
-          else if(referenceData instanceof java.util.Date) {
-            assertTrue("rowNum=" + numTuplesRead + " colNum=" + colPos + " Reference data=" + ((java.util.Date)referenceData).getTime() + " actual=" +
-                ((DateTime)t.get(colPos)).getMillis() + "; types=(" + referenceData.getClass() + "," + t.get(colPos).getClass() + ")",
+        for (Object referenceData : primitiveRows[numTuplesRead]) {
+          if (referenceData == null) {
+            assertTrue("rowNum=" + numTuplesRead + " colNum=" + colPos
+                + " Reference data is null; actual "
+                + t.get(colPos), t.get(colPos) == null);
+          } else if (referenceData instanceof java.util.Date) {
+            // Note that here we ignore nanos part of Hive Timestamp since nanos are dropped when
+            // reading Hive from Pig by design.
+            assertTrue("rowNum=" + numTuplesRead + " colNum=" + colPos
+                + " Reference data=" + ((java.util.Date)referenceData).getTime()
+                + " actual=" + ((DateTime)t.get(colPos)).getMillis()
+                + "; types=(" + referenceData.getClass() + "," + t.get(colPos).getClass() + ")",
                 ((java.util.Date)referenceData).getTime()== ((DateTime)t.get(colPos)).getMillis());
-            //note that here we ignore nanos part of Hive Timestamp since nanos are dropped when reading Hive from Pig by design
-          }
-          else {
-            assertTrue("rowNum=" + numTuplesRead + " colNum=" + colPos + " Reference data=" + referenceData + " actual=" +
-                t.get(colPos) + "; types=(" + referenceData.getClass() + "," + t.get(colPos).getClass() + ")",
+          } else {
+            // Doing String comps here as value objects in Hive in Pig are different so equals()
+            // doesn't work.
+            assertTrue("rowNum=" + numTuplesRead + " colNum=" + colPos
+                + " Reference data=" + referenceData + " actual=" + t.get(colPos)
+                + "; types=(" + referenceData.getClass() + "," + t.get(colPos).getClass() + ") ",
                 referenceData.toString().equals(t.get(colPos).toString()));
-            //doing String comps here as value objects in Hive in Pig are different so equals() doesn't work
           }
           colPos++;
         }
@@ -633,10 +679,10 @@ public class TestHCatLoader {
     }
     private static void setupAllTypesTable(Driver driver) throws Exception {
       String[] primitiveData = new String[primitiveRows.length];
-      for(int i = 0; i < primitiveRows.length; i++) {
+      for (int i = 0; i < primitiveRows.length; i++) {
         Object[] rowData = primitiveRows[i];
         StringBuilder row = new StringBuilder();
-        for(Object cell : rowData) {
+        for (Object cell : rowData) {
           row.append(row.length() == 0 ? "" : "\t").append(cell == null ? null : cell);
         }
         primitiveData[i] = row.toString();

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java?rev=1628660&r1=1628659&r2=1628660&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatLoaderComplexSchema.java Wed Oct  1 09:44:02 2014
@@ -18,19 +18,25 @@
  */
 package org.apache.hive.hcatalog.pig;
 
+import com.google.common.collect.ImmutableSet;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-
-import junit.framework.Assert;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -46,12 +52,20 @@ import org.apache.pig.impl.logicalLayer.
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
 
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assume.assumeTrue;
+
+@RunWith(Parameterized.class)
 public class TestHCatLoaderComplexSchema {
 
   //private static MiniCluster cluster = MiniCluster.buildCluster();
@@ -59,13 +73,33 @@ public class TestHCatLoaderComplexSchema
   //private static Properties props;
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatLoaderComplexSchema.class);
 
-  private void dropTable(String tablename) throws IOException, CommandNeedRetryException {
-    driver.run("drop table " + tablename);
+  private static final Map<String, Set<String>> DISABLED_STORAGE_FORMATS =
+      new HashMap<String, Set<String>>() {{
+        put(IOConstants.AVRO, new HashSet<String>() {{
+          add("testSyntheticComplexSchema");
+          add("testTupleInBagInTupleInBag");
+          add("testMapWithComplexData");
+        }});
+        put(IOConstants.PARQUETFILE, new HashSet<String>() {{
+          add("testSyntheticComplexSchema");
+          add("testTupleInBagInTupleInBag");
+          add("testMapWithComplexData");
+        }});
+      }};
+
+  private String storageFormat;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return StorageFormats.names();
+  }
+
+  public TestHCatLoaderComplexSchema(String storageFormat) {
+    this.storageFormat = storageFormat;
   }
 
-  protected String storageFormat() {
-    return "RCFILE tblproperties('hcat.isd'='org.apache.hive.hcatalog.rcfile.RCFileInputDriver'," +
-      "'hcat.osd'='org.apache.hive.hcatalog.rcfile.RCFileOutputDriver')";
+  private void dropTable(String tablename) throws IOException, CommandNeedRetryException {
+    driver.run("drop table " + tablename);
   }
 
   private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException {
@@ -74,7 +108,7 @@ public class TestHCatLoaderComplexSchema
     if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) {
       createTable = createTable + "partitioned by (" + partitionedBy + ") ";
     }
-    createTable = createTable + "stored as " + storageFormat();
+    createTable = createTable + "stored as " + storageFormat;
     LOG.info("Creating table:\n {}", createTable);
     CommandProcessorResponse result = driver.run(createTable);
     int retCode = result.getResponseCode();
@@ -89,7 +123,6 @@ public class TestHCatLoaderComplexSchema
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-
     HiveConf hiveConf = new HiveConf(TestHCatLoaderComplexSchema.class);
     hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
     hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
@@ -98,7 +131,6 @@ public class TestHCatLoaderComplexSchema
     SessionState.start(new CliSessionState(hiveConf));
     //props = new Properties();
     //props.setProperty("fs.default.name", cluster.getProperties().getProperty("fs.default.name"));
-
   }
 
   private static final TupleFactory tf = TupleFactory.getInstance();
@@ -118,6 +150,7 @@ public class TestHCatLoaderComplexSchema
    */
   @Test
   public void testSyntheticComplexSchema() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     String pigSchema =
         "a: " +
         "(" +
@@ -186,7 +219,6 @@ public class TestHCatLoaderComplexSchema
     verifyWriteRead("testSyntheticComplexSchema", pigSchema, tableSchema, data, false);
     verifyWriteRead("testSyntheticComplexSchema2", pigSchema, tableSchema2, data, true);
     verifyWriteRead("testSyntheticComplexSchema2", pigSchema, tableSchema2, data, false);
-
   }
 
   private void verifyWriteRead(String tablename, String pigSchema, String tableSchema, List<Tuple> data, boolean provideSchemaToStorer)
@@ -219,7 +251,7 @@ public class TestHCatLoaderComplexSchema
       }
       Schema dumpedXSchema = server.dumpSchema("X");
 
-      Assert.assertEquals(
+      assertEquals(
         "expected " + dumpedASchema + " but was " + dumpedXSchema + " (ignoring field names)",
         "",
         compareIgnoreFiledNames(dumpedASchema, dumpedXSchema));
@@ -230,14 +262,14 @@ public class TestHCatLoaderComplexSchema
   }
 
   private void compareTuples(Tuple t1, Tuple t2) throws ExecException {
-    Assert.assertEquals("Tuple Sizes don't match", t1.size(), t2.size());
+    assertEquals("Tuple Sizes don't match", t1.size(), t2.size());
     for (int i = 0; i < t1.size(); i++) {
       Object f1 = t1.get(i);
       Object f2 = t2.get(i);
-      Assert.assertNotNull("left", f1);
-      Assert.assertNotNull("right", f2);
+      assertNotNull("left", f1);
+      assertNotNull("right", f2);
       String msg = "right: " + f1 + ", left: " + f2;
-      Assert.assertEquals(msg, noOrder(f1.toString()), noOrder(f2.toString()));
+      assertEquals(msg, noOrder(f1.toString()), noOrder(f2.toString()));
     }
   }
 
@@ -278,6 +310,7 @@ public class TestHCatLoaderComplexSchema
    */
   @Test
   public void testTupleInBagInTupleInBag() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     String pigSchema = "a: { b : ( c: { d: (i : long) } ) }";
 
     String tableSchema = "a array< array< bigint > >";
@@ -297,11 +330,11 @@ public class TestHCatLoaderComplexSchema
 
     verifyWriteRead("TupleInBagInTupleInBag3", pigSchema, tableSchema2, data, true);
     verifyWriteRead("TupleInBagInTupleInBag4", pigSchema, tableSchema2, data, false);
-
   }
 
   @Test
   public void testMapWithComplexData() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     String pigSchema = "a: long, b: map[]";
     String tableSchema = "a bigint, b map<string, struct<aa:bigint, ab:string>>";
 
@@ -320,6 +353,5 @@ public class TestHCatLoaderComplexSchema
     }
     verifyWriteRead("testMapWithComplexData", pigSchema, tableSchema, data, true);
     verifyWriteRead("testMapWithComplexData2", pigSchema, tableSchema, data, false);
-
   }
 }

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorer.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorer.java?rev=1628660&r1=1628659&r2=1628660&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorer.java (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorer.java Wed Oct  1 09:44:02 2014
@@ -18,18 +18,27 @@
  */
 package org.apache.hive.hcatalog.pig;
 
+import com.google.common.collect.ImmutableSet;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 
 import org.apache.hive.hcatalog.HcatTestUtils;
@@ -47,20 +56,92 @@ import org.apache.pig.impl.util.LogUtils
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 
-import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
+
+@RunWith(Parameterized.class)
 public class TestHCatStorer extends HCatBaseTest {
   private static final Logger LOG = LoggerFactory.getLogger(TestHCatStorer.class);
 
   private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data";
 
+  private static final Map<String, Set<String>> DISABLED_STORAGE_FORMATS =
+    new HashMap<String, Set<String>>() {{
+      put(IOConstants.AVRO, new HashSet<String>() {{
+        add("testBagNStruct");
+        add("testDateCharTypes");
+        add("testDynamicPartitioningMultiPartColsInDataNoSpec");
+        add("testDynamicPartitioningMultiPartColsInDataPartialSpec");
+        add("testMultiPartColsInData");
+        add("testPartColsInData");
+        add("testStoreFuncAllSimpleTypes");
+        add("testStoreFuncSimple");
+        add("testStoreInPartiitonedTbl");
+        add("testStoreMultiTables");
+        add("testStoreWithNoCtorArgs");
+        add("testStoreWithNoSchema");
+        add("testWriteChar");
+        add("testWriteDate");
+        add("testWriteDate2");
+        add("testWriteDate3");
+        add("testWriteDecimal");
+        add("testWriteDecimalX");
+        add("testWriteDecimalXY");
+        add("testWriteSmallint");
+        add("testWriteTimestamp");
+        add("testWriteTinyint");
+        add("testWriteVarchar");
+      }});
+      put(IOConstants.PARQUETFILE, new HashSet<String>() {{
+        add("testBagNStruct");
+        add("testDateCharTypes");
+        add("testDynamicPartitioningMultiPartColsInDataNoSpec");
+        add("testDynamicPartitioningMultiPartColsInDataPartialSpec");
+        add("testMultiPartColsInData");
+        add("testPartColsInData");
+        add("testStoreFuncAllSimpleTypes");
+        add("testStoreFuncSimple");
+        add("testStoreInPartiitonedTbl");
+        add("testStoreMultiTables");
+        add("testStoreWithNoCtorArgs");
+        add("testStoreWithNoSchema");
+        add("testWriteChar");
+        add("testWriteDate");
+        add("testWriteDate2");
+        add("testWriteDate3");
+        add("testWriteDecimal");
+        add("testWriteDecimalX");
+        add("testWriteDecimalXY");
+        add("testWriteSmallint");
+        add("testWriteTimestamp");
+        add("testWriteTinyint");
+        add("testWriteVarchar");
+      }});
+    }};
+
+  private String storageFormat;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return StorageFormats.names();
+  }
+
+  public TestHCatStorer(String storageFormat) {
+    this.storageFormat = storageFormat;
+  }
+
   //Start: tests that check values from Pig that are out of range for target column
   @Test
   public void testWriteTinyint() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     pigValueRangeTest("junitTypeTest1", "tinyint", "int", null, Integer.toString(1), Integer.toString(1));
     pigValueRangeTestOverflow("junitTypeTest1", "tinyint", "int", null, Integer.toString(300));
     pigValueRangeTestOverflow("junitTypeTest2", "tinyint", "int", HCatBaseStorer.OOR_VALUE_OPT_VALUES.Null,
@@ -71,6 +152,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteSmallint() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     pigValueRangeTest("junitTypeTest1", "smallint", "int", null, Integer.toString(Short.MIN_VALUE),
       Integer.toString(Short.MIN_VALUE));
     pigValueRangeTestOverflow("junitTypeTest2", "smallint", "int", HCatBaseStorer.OOR_VALUE_OPT_VALUES.Null,
@@ -81,6 +163,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteChar() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     pigValueRangeTest("junitTypeTest1", "char(5)", "chararray", null, "xxx", "xxx  ");
     pigValueRangeTestOverflow("junitTypeTest1", "char(5)", "chararray", null, "too_long");
     pigValueRangeTestOverflow("junitTypeTest2", "char(5)", "chararray", HCatBaseStorer.OOR_VALUE_OPT_VALUES.Null,
@@ -91,6 +174,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteVarchar() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     pigValueRangeTest("junitTypeTest1", "varchar(5)", "chararray", null, "xxx", "xxx");
     pigValueRangeTestOverflow("junitTypeTest1", "varchar(5)", "chararray", null, "too_long");
     pigValueRangeTestOverflow("junitTypeTest2", "varchar(5)", "chararray", HCatBaseStorer.OOR_VALUE_OPT_VALUES.Null,
@@ -101,6 +185,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteDecimalXY() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     pigValueRangeTest("junitTypeTest1", "decimal(5,2)", "bigdecimal", null, BigDecimal.valueOf(1.2).toString(),
       BigDecimal.valueOf(1.2).toString());
     pigValueRangeTestOverflow("junitTypeTest1", "decimal(5,2)", "bigdecimal", null, BigDecimal.valueOf(12345.12).toString());
@@ -112,6 +197,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteDecimalX() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     //interestingly decimal(2) means decimal(2,0)
     pigValueRangeTest("junitTypeTest1", "decimal(2)", "bigdecimal", null, BigDecimal.valueOf(12).toString(),
       BigDecimal.valueOf(12).toString());
@@ -123,6 +209,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteDecimal() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     //decimal means decimal(10,0)
     pigValueRangeTest("junitTypeTest1", "decimal", "bigdecimal", null, BigDecimal.valueOf(1234567890).toString(),
       BigDecimal.valueOf(1234567890).toString());
@@ -137,8 +224,10 @@ public class TestHCatStorer extends HCat
    * include time to make sure it's 0
    */
   private static final String FORMAT_4_DATE = "yyyy-MM-dd HH:mm:ss";
+
   @Test
   public void testWriteDate() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     DateTime d = new DateTime(1991,10,11,0,0);
     pigValueRangeTest("junitTypeTest1", "date", "datetime", null, d.toString(),
       d.toString(FORMAT_4_DATE), FORMAT_4_DATE);
@@ -157,6 +246,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteDate3() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     DateTime d = new DateTime(1991,10,11,23,10,DateTimeZone.forOffsetHours(-11));
     FrontendException fe = null;
     //expect to fail since the time component is not 0
@@ -170,6 +260,7 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testWriteDate2() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     DateTime d = new DateTime(1991,11,12,0,0, DateTimeZone.forID("US/Eastern"));
     pigValueRangeTest("junitTypeTest1", "date", "datetime", null, d.toString(),
       d.toString(FORMAT_4_DATE), FORMAT_4_DATE);
@@ -193,6 +284,7 @@ public class TestHCatStorer extends HCat
    */
   @Test
   public void testWriteTimestamp() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     DateTime d = new DateTime(1991,10,11,14,23,30, 10);//uses default TZ
     pigValueRangeTest("junitTypeTest1", "timestamp", "datetime", null, d.toString(),
       d.toDateTime(DateTimeZone.getDefault()).toString());
@@ -229,13 +321,6 @@ public class TestHCatStorer extends HCat
   }
 
   /**
-   * this should be overridden in subclass to test with different file formats
-   */
-  String getStorageFormat() {
-    return "RCFILE";
-  }
-
-  /**
    * This is used to test how Pig values of various data types which are out of range for Hive target
    * column are handled.  Currently the options are to raise an error or write NULL.
    * 1. create a data file with 1 column, 1 row
@@ -258,7 +343,7 @@ public class TestHCatStorer extends HCat
     throws Exception {
     TestHCatLoader.dropTable(tblName, driver);
     final String field = "f1";
-    TestHCatLoader.createTable(tblName, field + " " + hiveType, null, driver, getStorageFormat());
+    TestHCatLoader.createTable(tblName, field + " " + hiveType, null, driver, storageFormat);
     HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, new String[] {inputValue});
     LOG.debug("File=" + INPUT_FILE_NAME);
     dumpFile(INPUT_FILE_NAME);
@@ -287,11 +372,11 @@ public class TestHCatStorer extends HCat
           //do nothing, fall through and verify the data
           break;
         case Throw:
-          Assert.assertTrue("Expected a FrontendException", fe != null);
-          Assert.assertEquals("Expected a different FrontendException.", fe.getMessage(), "Unable to store alias A");
+          assertTrue("Expected a FrontendException", fe != null);
+          assertEquals("Expected a different FrontendException.", fe.getMessage(), "Unable to store alias A");
           return;//this test is done
         default:
-          Assert.assertFalse("Unexpected goal: " + goal, 1 == 1);
+          assertFalse("Unexpected goal: " + goal, 1 == 1);
       }
     }
     logAndRegister(server, "B = load '" + tblName + "' using " + HCatLoader.class.getName() + "();", queryNumber);
@@ -310,17 +395,17 @@ public class TestHCatStorer extends HCat
       Tuple t = itr.next();
       if("date".equals(hiveType)) {
         DateTime dateTime = (DateTime)t.get(0);
-        Assert.assertTrue(format != null);
-        Assert.assertEquals("Comparing Pig to Raw data for table " + tblName, expectedValue, dateTime== null ? null : dateTime.toString(format));
+        assertTrue(format != null);
+        assertEquals("Comparing Pig to Raw data for table " + tblName, expectedValue, dateTime== null ? null : dateTime.toString(format));
       }
       else {
-        Assert.assertEquals("Comparing Pig to Raw data for table " + tblName, expectedValue, t.isNull(0) ? null : t.get(0).toString());
+        assertEquals("Comparing Pig to Raw data for table " + tblName, expectedValue, t.isNull(0) ? null : t.get(0).toString());
       }
       //see comment at "Dumping rows via SQL..." for why this doesn't work
-      //Assert.assertEquals("Comparing Pig to Hive", t.get(0), l.get(0));
+      //assertEquals("Comparing Pig to Hive", t.get(0), l.get(0));
       numRowsRead++;
     }
-    Assert.assertEquals("Expected " + 1 + " rows; got " + numRowsRead + " file=" + INPUT_FILE_NAME + "; table " +
+    assertEquals("Expected " + 1 + " rows; got " + numRowsRead + " file=" + INPUT_FILE_NAME + "; table " +
       tblName, 1, numRowsRead);
     /* Misc notes:
     Unfortunately Timestamp.toString() adjusts the value for local TZ and 't' is a String
@@ -334,10 +419,11 @@ public class TestHCatStorer extends HCat
    */
   @Test
   public void testDateCharTypes() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     final String tblName = "junit_date_char";
     TestHCatLoader.dropTable(tblName, driver);
     TestHCatLoader.createTable(tblName,
-      "id int, char5 char(5), varchar10 varchar(10), dec52 decimal(5,2)", null, driver, getStorageFormat());
+      "id int, char5 char(5), varchar10 varchar(10), dec52 decimal(5,2)", null, driver, storageFormat);
     int NUM_ROWS = 5;
     String[] rows = new String[NUM_ROWS];
     for(int i = 0; i < NUM_ROWS; i++) {
@@ -376,12 +462,12 @@ public class TestHCatStorer extends HCat
         rowFromPig.append(t.get(i)).append("\t");
       }
       rowFromPig.setLength(rowFromPig.length() - 1);
-      Assert.assertEquals("Comparing Pig to Raw data", rows[numRowsRead], rowFromPig.toString());
+      assertEquals("Comparing Pig to Raw data", rows[numRowsRead], rowFromPig.toString());
       //see comment at "Dumping rows via SQL..." for why this doesn't work (for all types)
-      //Assert.assertEquals("Comparing Pig to Hive", rowFromPig.toString(), l.get(numRowsRead));
+      //assertEquals("Comparing Pig to Hive", rowFromPig.toString(), l.get(numRowsRead));
       numRowsRead++;
     }
-    Assert.assertEquals("Expected " + NUM_ROWS + " rows; got " + numRowsRead + " file=" + INPUT_FILE_NAME, NUM_ROWS, numRowsRead);
+    assertEquals("Expected " + NUM_ROWS + " rows; got " + numRowsRead + " file=" + INPUT_FILE_NAME, NUM_ROWS, numRowsRead);
   }
 
   static void dumpFile(String fileName) throws Exception {
@@ -397,9 +483,10 @@ public class TestHCatStorer extends HCat
 
   @Test
   public void testPartColsInData() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int) partitioned by (b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int) partitioned by (b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -420,22 +507,23 @@ public class TestHCatStorer extends HCat
 
     while (itr.hasNext()) {
       Tuple t = itr.next();
-      Assert.assertEquals(2, t.size());
-      Assert.assertEquals(t.get(0), i);
-      Assert.assertEquals(t.get(1), "1");
+      assertEquals(2, t.size());
+      assertEquals(t.get(0), i);
+      assertEquals(t.get(1), "1");
       i++;
     }
 
-    Assert.assertFalse(itr.hasNext());
-    Assert.assertEquals(LOOP_SIZE, i);
+    assertFalse(itr.hasNext());
+    assertEquals(LOOP_SIZE, i);
   }
 
   @Test
   public void testMultiPartColsInData() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table employee");
     String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " +
-      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + getStorageFormat();
+      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + storageFormat;
 
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
@@ -464,20 +552,21 @@ public class TestHCatStorer extends HCat
     driver.run("select * from employee");
     ArrayList<String> results = new ArrayList<String>();
     driver.getResults(results);
-    Assert.assertEquals(4, results.size());
+    assertEquals(4, results.size());
     Collections.sort(results);
-    Assert.assertEquals(inputData[0], results.get(0));
-    Assert.assertEquals(inputData[1], results.get(1));
-    Assert.assertEquals(inputData[2], results.get(2));
-    Assert.assertEquals(inputData[3], results.get(3));
+    assertEquals(inputData[0], results.get(0));
+    assertEquals(inputData[1], results.get(1));
+    assertEquals(inputData[2], results.get(2));
+    assertEquals(inputData[3], results.get(3));
     driver.run("drop table employee");
   }
 
   @Test
   public void testStoreInPartiitonedTbl() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int) partitioned by (b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int) partitioned by (b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -498,20 +587,21 @@ public class TestHCatStorer extends HCat
 
     while (itr.hasNext()) {
       Tuple t = itr.next();
-      Assert.assertEquals(2, t.size());
-      Assert.assertEquals(t.get(0), i);
-      Assert.assertEquals(t.get(1), "1");
+      assertEquals(2, t.size());
+      assertEquals(t.get(0), i);
+      assertEquals(t.get(1), "1");
       i++;
     }
 
-    Assert.assertFalse(itr.hasNext());
-    Assert.assertEquals(11, i);
+    assertFalse(itr.hasNext());
+    assertEquals(11, i);
   }
 
   @Test
   public void testNoAlias() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     driver.run("drop table junit_parted");
-    String createTable = "create table junit_parted(a int, b string) partitioned by (ds string) stored as " + getStorageFormat();
+    String createTable = "create table junit_parted(a int, b string) partitioned by (ds string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -526,12 +616,12 @@ public class TestHCatStorer extends HCat
       server.executeBatch();
     } catch (PigException fe) {
       PigException pe = LogUtils.getPigException(fe);
-      Assert.assertTrue(pe instanceof FrontendException);
-      Assert.assertEquals(PigHCatUtil.PIG_EXCEPTION_CODE, pe.getErrorCode());
-      Assert.assertTrue(pe.getMessage().contains("Column name for a field is not specified. Please provide the full schema as an argument to HCatStorer."));
+      assertTrue(pe instanceof FrontendException);
+      assertEquals(PigHCatUtil.PIG_EXCEPTION_CODE, pe.getErrorCode());
+      assertTrue(pe.getMessage().contains("Column name for a field is not specified. Please provide the full schema as an argument to HCatStorer."));
       errCaught = true;
     }
-    Assert.assertTrue(errCaught);
+    assertTrue(errCaught);
     errCaught = false;
     try {
       server.setBatchOn();
@@ -541,20 +631,21 @@ public class TestHCatStorer extends HCat
       server.executeBatch();
     } catch (PigException fe) {
       PigException pe = LogUtils.getPigException(fe);
-      Assert.assertTrue(pe instanceof FrontendException);
-      Assert.assertEquals(PigHCatUtil.PIG_EXCEPTION_CODE, pe.getErrorCode());
-      Assert.assertTrue(pe.getMessage().contains("Column names should all be in lowercase. Invalid name found: B"));
+      assertTrue(pe instanceof FrontendException);
+      assertEquals(PigHCatUtil.PIG_EXCEPTION_CODE, pe.getErrorCode());
+      assertTrue(pe.getMessage().contains("Column names should all be in lowercase. Invalid name found: B"));
       errCaught = true;
     }
     driver.run("drop table junit_parted");
-    Assert.assertTrue(errCaught);
+    assertTrue(errCaught);
   }
 
   @Test
   public void testStoreMultiTables() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int, b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int, b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -598,18 +689,19 @@ public class TestHCatStorer extends HCat
 
     Iterator<String> itr = res.iterator();
     for (int i = 0; i < LOOP_SIZE * LOOP_SIZE; i++) {
-      Assert.assertEquals(input[i], itr.next());
+      assertEquals(input[i], itr.next());
     }
 
-    Assert.assertFalse(itr.hasNext());
+    assertFalse(itr.hasNext());
 
   }
 
   @Test
   public void testStoreWithNoSchema() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int, b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int, b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -637,18 +729,19 @@ public class TestHCatStorer extends HCat
     driver.run("drop table junit_unparted");
     Iterator<String> itr = res.iterator();
     for (int i = 0; i < LOOP_SIZE * LOOP_SIZE; i++) {
-      Assert.assertEquals(input[i], itr.next());
+      assertEquals(input[i], itr.next());
     }
 
-    Assert.assertFalse(itr.hasNext());
+    assertFalse(itr.hasNext());
 
   }
 
   @Test
   public void testStoreWithNoCtorArgs() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int, b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int, b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -676,18 +769,19 @@ public class TestHCatStorer extends HCat
     driver.run("drop table junit_unparted");
     Iterator<String> itr = res.iterator();
     for (int i = 0; i < LOOP_SIZE * LOOP_SIZE; i++) {
-      Assert.assertEquals(input[i], itr.next());
+      assertEquals(input[i], itr.next());
     }
 
-    Assert.assertFalse(itr.hasNext());
+    assertFalse(itr.hasNext());
 
   }
 
   @Test
   public void testEmptyStore() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int, b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int, b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -715,15 +809,16 @@ public class TestHCatStorer extends HCat
     driver.getResults(res);
     driver.run("drop table junit_unparted");
     Iterator<String> itr = res.iterator();
-    Assert.assertFalse(itr.hasNext());
+    assertFalse(itr.hasNext());
 
   }
 
   @Test
   public void testBagNStruct() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     driver.run("drop table junit_unparted");
     String createTable = "create table junit_unparted(b string,a struct<a1:int>,  arr_of_struct array<string>, " +
-      "arr_of_struct2 array<struct<s1:string,s2:string>>,  arr_of_struct3 array<struct<s3:string>>) stored as " + getStorageFormat();
+      "arr_of_struct2 array<struct<s1:string,s2:string>>,  arr_of_struct3 array<struct<s3:string>>) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -746,17 +841,18 @@ public class TestHCatStorer extends HCat
     driver.getResults(res);
     driver.run("drop table junit_unparted");
     Iterator<String> itr = res.iterator();
-    Assert.assertEquals("zookeeper\t{\"a1\":2}\t[\"pig\"]\t[{\"s1\":\"pnuts\",\"s2\":\"hdfs\"}]\t[{\"s3\":\"hadoop\"},{\"s3\":\"hcat\"}]", itr.next());
-    Assert.assertEquals("chubby\t{\"a1\":2}\t[\"sawzall\"]\t[{\"s1\":\"bigtable\",\"s2\":\"gfs\"}]\t[{\"s3\":\"mapreduce\"},{\"s3\":\"hcat\"}]", itr.next());
-    Assert.assertFalse(itr.hasNext());
+    assertEquals("zookeeper\t{\"a1\":2}\t[\"pig\"]\t[{\"s1\":\"pnuts\",\"s2\":\"hdfs\"}]\t[{\"s3\":\"hadoop\"},{\"s3\":\"hcat\"}]", itr.next());
+    assertEquals("chubby\t{\"a1\":2}\t[\"sawzall\"]\t[{\"s1\":\"bigtable\",\"s2\":\"gfs\"}]\t[{\"s3\":\"mapreduce\"},{\"s3\":\"hcat\"}]", itr.next());
+    assertFalse(itr.hasNext());
 
   }
 
   @Test
   public void testStoreFuncAllSimpleTypes() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int, b float, c double, d bigint, e string, h boolean, f binary, g binary) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int, b float, c double, d bigint, e string, h boolean, f binary, g binary) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -783,10 +879,10 @@ public class TestHCatStorer extends HCat
 
     Iterator<String> itr = res.iterator();
     String next = itr.next();
-    Assert.assertEquals("0\tNULL\tNULL\tNULL\tNULL\tNULL\tNULL\tNULL", next );
-    Assert.assertEquals("NULL\t4.2\t2.2\t4\tlets hcat\ttrue\tbinary-data\tNULL", itr.next());
-    Assert.assertEquals("3\t6.2999997\t3.3000000000000003\t6\tlets hcat\tfalse\tbinary-data\tNULL", itr.next());
-    Assert.assertFalse(itr.hasNext());
+    assertEquals("0\tNULL\tNULL\tNULL\tNULL\tNULL\tNULL\tNULL", next );
+    assertEquals("NULL\t4.2\t2.2\t4\tlets hcat\ttrue\tbinary-data\tNULL", itr.next());
+    assertEquals("3\t6.2999997\t3.3000000000000003\t6\tlets hcat\tfalse\tbinary-data\tNULL", itr.next());
+    assertFalse(itr.hasNext());
 
     server.registerQuery("B = load 'junit_unparted' using " + HCatLoader.class.getName() + ";");
     Iterator<Tuple> iter = server.openIterator("B");
@@ -797,21 +893,22 @@ public class TestHCatStorer extends HCat
       if (t.get(6) == null) {
         num5nulls++;
       } else {
-        Assert.assertTrue(t.get(6) instanceof DataByteArray);
+        assertTrue(t.get(6) instanceof DataByteArray);
       }
-      Assert.assertNull(t.get(7));
+      assertNull(t.get(7));
       count++;
     }
-    Assert.assertEquals(3, count);
-    Assert.assertEquals(1, num5nulls);
+    assertEquals(3, count);
+    assertEquals(1, num5nulls);
     driver.run("drop table junit_unparted");
   }
 
   @Test
   public void testStoreFuncSimple() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table junit_unparted");
-    String createTable = "create table junit_unparted(a int, b string) stored as " + getStorageFormat();
+    String createTable = "create table junit_unparted(a int, b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -841,19 +938,20 @@ public class TestHCatStorer extends HCat
     for (int i = 1; i <= LOOP_SIZE; i++) {
       String si = i + "";
       for (int j = 1; j <= LOOP_SIZE; j++) {
-        Assert.assertEquals(si + "\t" + j, itr.next());
+        assertEquals(si + "\t" + j, itr.next());
       }
     }
-    Assert.assertFalse(itr.hasNext());
+    assertFalse(itr.hasNext());
 
   }
 
   @Test
   public void testDynamicPartitioningMultiPartColsInDataPartialSpec() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table if exists employee");
     String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " +
-      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + getStorageFormat();
+      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + storageFormat;
 
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
@@ -876,21 +974,22 @@ public class TestHCatStorer extends HCat
     driver.run("select * from employee");
     ArrayList<String> results = new ArrayList<String>();
     driver.getResults(results);
-    Assert.assertEquals(4, results.size());
+    assertEquals(4, results.size());
     Collections.sort(results);
-    Assert.assertEquals(inputData[0], results.get(0));
-    Assert.assertEquals(inputData[1], results.get(1));
-    Assert.assertEquals(inputData[2], results.get(2));
-    Assert.assertEquals(inputData[3], results.get(3));
+    assertEquals(inputData[0], results.get(0));
+    assertEquals(inputData[1], results.get(1));
+    assertEquals(inputData[2], results.get(2));
+    assertEquals(inputData[3], results.get(3));
     driver.run("drop table employee");
   }
 
   @Test
   public void testDynamicPartitioningMultiPartColsInDataNoSpec() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table if exists employee");
     String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " +
-      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + getStorageFormat();
+      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + storageFormat;
 
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
@@ -913,21 +1012,22 @@ public class TestHCatStorer extends HCat
     driver.run("select * from employee");
     ArrayList<String> results = new ArrayList<String>();
     driver.getResults(results);
-    Assert.assertEquals(4, results.size());
+    assertEquals(4, results.size());
     Collections.sort(results);
-    Assert.assertEquals(inputData[0], results.get(0));
-    Assert.assertEquals(inputData[1], results.get(1));
-    Assert.assertEquals(inputData[2], results.get(2));
-    Assert.assertEquals(inputData[3], results.get(3));
+    assertEquals(inputData[0], results.get(0));
+    assertEquals(inputData[1], results.get(1));
+    assertEquals(inputData[2], results.get(2));
+    assertEquals(inputData[3], results.get(3));
     driver.run("drop table employee");
   }
 
   @Test
   public void testDynamicPartitioningMultiPartColsNoDataInDataNoSpec() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table if exists employee");
     String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " +
-      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + getStorageFormat();
+      " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS " + storageFormat;
 
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
@@ -947,15 +1047,16 @@ public class TestHCatStorer extends HCat
     driver.run("select * from employee");
     ArrayList<String> results = new ArrayList<String>();
     driver.getResults(results);
-    Assert.assertEquals(0, results.size());
+    assertEquals(0, results.size());
     driver.run("drop table employee");
   }
+
   @Test
-  public void testPartitionPublish()
-    throws IOException, CommandNeedRetryException {
+  public void testPartitionPublish() throws IOException, CommandNeedRetryException {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
 
     driver.run("drop table ptn_fail");
-    String createTable = "create table ptn_fail(a int, c string) partitioned by (b string) stored as " + getStorageFormat();
+    String createTable = "create table ptn_fail(a int, c string) partitioned by (b string) stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table.");
@@ -987,11 +1088,11 @@ public class TestHCatStorer extends HCat
 
     ArrayList<String> res = new ArrayList<String>();
     driver.getResults(res);
-    Assert.assertEquals(0, res.size());
+    assertEquals(0, res.size());
 
     // Make sure the partitions directory is not in hdfs.
-    Assert.assertTrue((new File(TEST_WAREHOUSE_DIR + "/ptn_fail")).exists());
-    Assert.assertFalse((new File(TEST_WAREHOUSE_DIR + "/ptn_fail/b=math"))
+    assertTrue((new File(TEST_WAREHOUSE_DIR + "/ptn_fail")).exists());
+    assertFalse((new File(TEST_WAREHOUSE_DIR + "/ptn_fail/b=math"))
       .exists());
   }
 

Modified: hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java?rev=1628660&r1=1628659&r2=1628660&view=diff
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java (original)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestHCatStorerMulti.java Wed Oct  1 09:44:02 2014
@@ -18,18 +18,25 @@
  */
 package org.apache.hive.hcatalog.pig;
 
+import com.google.common.collect.ImmutableSet;
+
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.io.IOConstants;
+import org.apache.hadoop.hive.ql.io.StorageFormats;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 import org.apache.hive.hcatalog.common.HCatUtil;
@@ -41,13 +48,17 @@ import org.apache.pig.PigServer;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assume.assumeTrue;
 
+@RunWith(Parameterized.class)
 public class TestHCatStorerMulti {
   public static final String TEST_DATA_DIR = HCatUtil.makePathASafeFileName(
-          System.getProperty("user.dir") + "/build/test/data/" +
-                  TestHCatStorerMulti.class.getCanonicalName() + "-" + System.currentTimeMillis());
+      System.getProperty("user.dir") + "/build/test/data/" +
+          TestHCatStorerMulti.class.getCanonicalName() + "-" + System.currentTimeMillis());
   private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
   private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data";
 
@@ -57,9 +68,29 @@ public class TestHCatStorerMulti {
 
   private static Map<Integer, Pair<Integer, String>> basicInputData;
 
-  protected String storageFormat() {
-    return "RCFILE tblproperties('hcat.isd'='org.apache.hive.hcatalog.rcfile.RCFileInputDriver'," +
-      "'hcat.osd'='org.apache.hive.hcatalog.rcfile.RCFileOutputDriver')";
+  private static final Map<String, Set<String>> DISABLED_STORAGE_FORMATS =
+      new HashMap<String, Set<String>>() {{
+        put(IOConstants.AVRO, new HashSet<String>() {{
+          add("testStoreBasicTable");
+          add("testStorePartitionedTable");
+          add("testStoreTableMulti");
+        }});
+        put(IOConstants.PARQUETFILE, new HashSet<String>() {{
+          add("testStoreBasicTable");
+          add("testStorePartitionedTable");
+          add("testStoreTableMulti");
+        }});
+      }};
+
+  private String storageFormat;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> generateParameters() {
+    return StorageFormats.names();
+  }
+
+  public TestHCatStorerMulti(String storageFormat) {
+    this.storageFormat = storageFormat;
   }
 
   private void dropTable(String tablename) throws IOException, CommandNeedRetryException {
@@ -72,7 +103,7 @@ public class TestHCatStorerMulti {
     if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) {
       createTable = createTable + "partitioned by (" + partitionedBy + ") ";
     }
-    createTable = createTable + "stored as " + storageFormat();
+    createTable = createTable + "stored as " + storageFormat;
     int retCode = driver.run(createTable).getResponseCode();
     if (retCode != 0) {
       throw new IOException("Failed to create table. [" + createTable + "], return code from hive driver : [" + retCode + "]");
@@ -85,6 +116,8 @@ public class TestHCatStorerMulti {
 
   @Before
   public void setUp() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
+
     if (driver == null) {
       HiveConf hiveConf = new HiveConf(this.getClass());
       hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
@@ -105,6 +138,7 @@ public class TestHCatStorerMulti {
 
   @Test
   public void testStoreBasicTable() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     createTable(BASIC_TABLE, "a int, b string");
 
     populateBasicFile();
@@ -124,6 +158,7 @@ public class TestHCatStorerMulti {
 
   @Test
   public void testStorePartitionedTable() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     createTable(PARTITIONED_TABLE, "a int, b string", "bkt string");
 
     populateBasicFile();
@@ -147,6 +182,7 @@ public class TestHCatStorerMulti {
 
   @Test
   public void testStoreTableMulti() throws Exception {
+    assumeTrue(!TestUtil.shouldSkip(storageFormat, DISABLED_STORAGE_FORMATS));
     createTable(BASIC_TABLE, "a int, b string");
     createTable(PARTITIONED_TABLE, "a int, b string", "bkt string");
 

Added: hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestUtil.java?rev=1628660&view=auto
==============================================================================
--- hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestUtil.java (added)
+++ hive/trunk/hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hive/hcatalog/pig/TestUtil.java Wed Oct  1 09:44:02 2014
@@ -0,0 +1,76 @@
+/**
+ * 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.hive.hcatalog.pig;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Test utilities for selectively disabling specific test methods for given storage formats.
+ */
+public class TestUtil {
+  private static final Map<String, Set<String>> SAMPLE_DISABLED_TESTS_MAP =
+      new HashMap<String, Set<String>>() {{
+        put("test", new HashSet<String>() {{
+          add("testShouldSkip");
+        }});
+      }};
+
+  /**
+   * Determine whether the caller test method is in a set of disabled test methods for a given
+   * storage format.
+   *
+   * @param storageFormat The name of the storage format used in a STORED AS clause.
+   * @param disabledTestsMap Map of storage format name to set of test method names that indicate
+   *        which test methods should not run against the given storage format.
+   * @return True if the caller test method should be skipped for the given storage format.
+   */
+  public static boolean shouldSkip(String storageFormat, Map<String, Set<String>> disabledTestsMap) {
+    final StackTraceElement[] elements = Thread.currentThread().getStackTrace();
+    // The "bottom" of the call stack is at the front of the array. The elements are as follows:
+    //   [0] getStackTrace()
+    //   [1] shouldSkip()
+    //   [2] caller test method
+    String methodName = elements[2].getMethodName();
+    if (!disabledTestsMap.containsKey(storageFormat)) {
+      return false;
+    }
+
+    Set<String> disabledMethods = disabledTestsMap.get(storageFormat);
+    return disabledMethods.contains(methodName);
+  }
+
+  @Test
+  public void testShouldSkip() {
+    assertTrue(TestUtil.shouldSkip("test", SAMPLE_DISABLED_TESTS_MAP));
+  }
+
+  @Test
+  public void testShouldNotSkip() {
+    assertFalse(TestUtil.shouldSkip("test", SAMPLE_DISABLED_TESTS_MAP));
+    assertFalse(TestUtil.shouldSkip("foo", SAMPLE_DISABLED_TESTS_MAP));
+  }
+}

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java?rev=1628660&r1=1628659&r2=1628660&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/StorageFormats.java Wed Oct  1 09:44:02 2014
@@ -90,7 +90,9 @@ public class StorageFormats {
    * includes both native Hive storage formats as well as those enumerated in the
    * ADDITIONAL_STORAGE_FORMATS table.
    *
-   * @return List of storage format as paramters.
+   * @return List of storage format as a Collection of Object arrays, each containing (in order):
+   *         Storage format name, SerDe class name, InputFormat class name, OutputFormat class name.
+   *         This list is used as the parameters to JUnit parameterized tests.
    */
   public static Collection<Object[]> asParameters() {
     List<Object[]> parameters = new ArrayList<Object[]>();
@@ -130,5 +132,21 @@ public class StorageFormats {
 
     return parameters;
   }
+
+  /**
+   * Returns a list of the names of storage formats.
+   *
+   * @return List of names of storage formats.
+   */
+  public static Collection<Object[]> names() {
+    List<Object[]> names = new ArrayList<Object[]>();
+    for (StorageFormatDescriptor descriptor : ServiceLoader.load(StorageFormatDescriptor.class)) {
+      String[] formatNames = new String[descriptor.getNames().size()];
+      formatNames = descriptor.getNames().toArray(formatNames);
+      String[] params = { formatNames[0] };
+      names.add(params);
+    }
+    return names;
+  }
 }