You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2014/10/14 21:07:05 UTC

svn commit: r1631841 [18/42] - in /hive/branches/llap: ./ accumulo-handler/ accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/columns/ accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/mr/ accumulo-handler/src/java/org/apache/hadoop/hiv...

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java Tue Oct 14 19:06:45 2014
@@ -335,6 +335,104 @@ public class TestNewIntegerEncoding {
   }
 
   @Test
+  public void testDeltaOverflow() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory
+          .getReflectionObjectInspector(Long.class,
+              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[]{4513343538618202719l, 4513343538618202711l,
+        2911390882471569739l,
+        -9181829309989854913l};
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    for (Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile
+        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testDeltaOverflow2() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory
+          .getReflectionObjectInspector(Long.class,
+              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[]{Long.MAX_VALUE, 4513343538618202711l,
+        2911390882471569739l,
+        Long.MIN_VALUE};
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    for (Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile
+        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testDeltaOverflow3() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory
+          .getReflectionObjectInspector(Long.class,
+              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[]{-4513343538618202711l, -2911390882471569739l, -2,
+        Long.MAX_VALUE};
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    for (Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile
+        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
   public void testIntegerMin() throws Exception {
     ObjectInspector inspector;
     synchronized (TestOrcFile.class) {

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java Tue Oct 14 19:06:45 2014
@@ -1754,9 +1754,9 @@ public class TestOrcFile {
           stripe.getDataLength() < 5000);
     }
     // with HIVE-7832, the dictionaries will be disabled after writing the first
-    // stripe as there are too many distinct values. Hence only 3 stripes as
+    // stripe as there are too many distinct values. Hence only 4 stripes as
     // compared to 25 stripes in version 0.11 (above test case)
-    assertEquals(3, i);
+    assertEquals(4, i);
     assertEquals(2500, reader.getNumberOfRows());
   }
 

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java Tue Oct 14 19:06:45 2014
@@ -56,6 +56,7 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 
 public class TestOrcRawRecordMerger {
@@ -574,12 +575,14 @@ public class TestOrcRawRecordMerger {
         OrcRecordUpdater.getOperation(event));
     assertEquals(new ReaderKey(0, BUCKET, 0, 200), id);
     assertEquals("update 1", getValue(event));
+    assertFalse(merger.isDelete(event));
 
     assertEquals(true, merger.next(id, event));
     assertEquals(OrcRecordUpdater.INSERT_OPERATION,
         OrcRecordUpdater.getOperation(event));
     assertEquals(new ReaderKey(0, BUCKET, 1, 0), id);
     assertEquals("second", getValue(event));
+    assertFalse(merger.isDelete(event));
 
     assertEquals(true, merger.next(id, event));
     assertEquals(OrcRecordUpdater.UPDATE_OPERATION,
@@ -616,6 +619,7 @@ public class TestOrcRawRecordMerger {
         OrcRecordUpdater.getOperation(event));
     assertEquals(new ReaderKey(0, BUCKET, 7, 200), id);
     assertNull(OrcRecordUpdater.getRow(event));
+    assertTrue(merger.isDelete(event));
 
     assertEquals(true, merger.next(id, event));
     assertEquals(OrcRecordUpdater.DELETE_OPERATION,

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestSerializationUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestSerializationUtils.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestSerializationUtils.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestSerializationUtils.java Tue Oct 14 19:06:45 2014
@@ -17,15 +17,18 @@
  */
 package org.apache.hadoop.hive.ql.io.orc;
 
-import org.junit.Test;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.InputStream;
 import java.math.BigInteger;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
+import org.junit.Test;
+
+import com.google.common.math.LongMath;
 
 public class TestSerializationUtils {
 
@@ -112,6 +115,47 @@ public class TestSerializationUtils {
         SerializationUtils.readBigInteger(fromBuffer(buffer)));
   }
 
+  @Test
+  public void testSubtractionOverflow() {
+    // cross check results with Guava results below
+    SerializationUtils utils = new SerializationUtils();
+    assertEquals(false, utils.isSafeSubtract(22222222222L, Long.MIN_VALUE));
+    assertEquals(false, utils.isSafeSubtract(-22222222222L, Long.MAX_VALUE));
+    assertEquals(false, utils.isSafeSubtract(Long.MIN_VALUE, Long.MAX_VALUE));
+    assertEquals(true, utils.isSafeSubtract(-1553103058346370095L, 6553103058346370095L));
+    assertEquals(true, utils.isSafeSubtract(0, Long.MAX_VALUE));
+    assertEquals(true, utils.isSafeSubtract(Long.MIN_VALUE, 0));
+  }
+
+  @Test
+  public void testSubtractionOverflowGuava() {
+    try {
+      LongMath.checkedSubtract(22222222222L, Long.MIN_VALUE);
+      fail("expected ArithmeticException for overflow");
+    } catch (ArithmeticException ex) {
+      assertEquals(ex.getMessage(), "overflow");
+    }
+
+    try {
+      LongMath.checkedSubtract(-22222222222L, Long.MAX_VALUE);
+      fail("expected ArithmeticException for overflow");
+    } catch (ArithmeticException ex) {
+      assertEquals(ex.getMessage(), "overflow");
+    }
+
+    try {
+      LongMath.checkedSubtract(Long.MIN_VALUE, Long.MAX_VALUE);
+      fail("expected ArithmeticException for overflow");
+    } catch (ArithmeticException ex) {
+      assertEquals(ex.getMessage(), "overflow");
+    }
+
+    assertEquals(-8106206116692740190L,
+        LongMath.checkedSubtract(-1553103058346370095L, 6553103058346370095L));
+    assertEquals(-Long.MAX_VALUE, LongMath.checkedSubtract(0, Long.MAX_VALUE));
+    assertEquals(Long.MIN_VALUE, LongMath.checkedSubtract(Long.MIN_VALUE, 0));
+  }
+
   public static void main(String[] args) throws Exception {
     TestSerializationUtils test = new TestSerializationUtils();
     test.testDoubles();

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/zookeeper/TestZookeeperLockManager.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/zookeeper/TestZookeeperLockManager.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/zookeeper/TestZookeeperLockManager.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/zookeeper/TestZookeeperLockManager.java Tue Oct 14 19:06:45 2014
@@ -25,6 +25,7 @@ import java.util.Collections;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode;
 import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject;
+import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
 import org.junit.Assert;
@@ -87,14 +88,14 @@ public class TestZookeeperLockManager {
   public void testGetQuorumServers() {
     conf.setVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_QUORUM, "node1");
     conf.setVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CLIENT_PORT, "9999");
-    Assert.assertEquals("node1:9999", ZooKeeperHiveLockManager.getQuorumServers(conf));
+    Assert.assertEquals("node1:9999", ZooKeeperHiveHelper.getQuorumServers(conf));
 
     conf.setVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_QUORUM, "node1,node2,node3");
     conf.setVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CLIENT_PORT, "9999");
-    Assert.assertEquals("node1:9999,node2:9999,node3:9999", ZooKeeperHiveLockManager.getQuorumServers(conf));
+    Assert.assertEquals("node1:9999,node2:9999,node3:9999", ZooKeeperHiveHelper.getQuorumServers(conf));
 
     conf.setVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_QUORUM, "node1:5666,node2,node3");
     conf.setVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_CLIENT_PORT, "9999");
-    Assert.assertEquals("node1:5666,node2:9999,node3:9999", ZooKeeperHiveLockManager.getQuorumServers(conf));
+    Assert.assertEquals("node1:5666,node2:9999,node3:9999", ZooKeeperHiveHelper.getQuorumServers(conf));
   }
 }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java Tue Oct 14 19:06:45 2014
@@ -21,14 +21,18 @@ package org.apache.hadoop.hive.ql.metada
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.regex.Pattern;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
@@ -45,6 +49,7 @@ import org.apache.hadoop.hive.serde.serd
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer;
 import org.apache.hadoop.hive.serde2.thrift.test.Complex;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
@@ -63,6 +68,9 @@ public class TestHive extends TestCase {
   protected void setUp() throws Exception {
     super.setUp();
     hiveConf = new HiveConf(this.getClass());
+    // enable trash so it can be tested
+    hiveConf.setFloat("fs.trash.checkpoint.interval", 30);  // FS_TRASH_CHECKPOINT_INTERVAL_KEY (hadoop-2)
+    hiveConf.setFloat("fs.trash.interval", 30);             // FS_TRASH_INTERVAL_KEY (hadoop-2)
     SessionState.start(hiveConf);
     try {
       hm = Hive.get(hiveConf);
@@ -79,6 +87,9 @@ public class TestHive extends TestCase {
   protected void tearDown() throws Exception {
     try {
       super.tearDown();
+      // disable trash
+      hiveConf.setFloat("fs.trash.checkpoint.interval", 30);  // FS_TRASH_CHECKPOINT_INTERVAL_KEY (hadoop-2)
+      hiveConf.setFloat("fs.trash.interval", 30);             // FS_TRASH_INTERVAL_KEY (hadoop-2)
       Hive.closeCurrent();
     } catch (Exception e) {
       System.err.println(StringUtils.stringifyException(e));
@@ -294,7 +305,7 @@ public class TestHive extends TestCase {
     try {
       String dbName = "db_for_testgettables";
       String table1Name = "table1";
-      hm.dropDatabase(dbName, true, true);
+      hm.dropDatabase(dbName, true, true, true);
 
       Database db = new Database();
       db.setName(dbName);
@@ -330,16 +341,92 @@ public class TestHive extends TestCase {
 
       // Drop all tables
       for (String tableName : hm.getAllTables(dbName)) {
+        Table table = hm.getTable(dbName, tableName);
         hm.dropTable(dbName, tableName);
+        assertFalse(fs.exists(table.getPath()));
       }
       hm.dropDatabase(dbName);
     } catch (Throwable e) {
       System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testGetTables() failed");
+      System.err.println("testGetAndDropTables() failed");
       throw e;
     }
   }
 
+  public void testDropTableTrash() throws Throwable {
+    if (!ShimLoader.getHadoopShims().supportTrashFeature()) {
+      return; // it's hadoop-1
+    }
+    try {
+      String dbName = "db_for_testdroptable";
+      hm.dropDatabase(dbName, true, true, true);
+
+      Database db = new Database();
+      db.setName(dbName);
+      hm.createDatabase(db);
+
+      List<String> ts = new ArrayList<String>(2);
+      String tableBaseName = "droptable";
+      ts.add(tableBaseName + "1");
+      ts.add(tableBaseName + "2");
+      Table tbl1 = createTestTable(dbName, ts.get(0));
+      hm.createTable(tbl1);
+      Table tbl2 = createTestTable(dbName, ts.get(1));
+      hm.createTable(tbl2);
+      // test dropping tables and trash behavior
+      Table table1 = hm.getTable(dbName, ts.get(0));
+      assertNotNull(table1);
+      assertEquals(ts.get(0), table1.getTableName());
+      Path path1 = table1.getPath();
+      FileSystem fs = path1.getFileSystem(hiveConf);
+      assertTrue(fs.exists(path1));
+      // drop table and check that trash works
+      Path trashDir = ShimLoader.getHadoopShims().getCurrentTrashPath(hiveConf, fs);
+      assertNotNull("trash directory should not be null", trashDir);
+      Path trash1 = mergePaths(trashDir, path1);
+      Path pathglob = trash1.suffix("*");;
+      FileStatus before[] = fs.globStatus(pathglob);
+      hm.dropTable(dbName, ts.get(0));
+      assertFalse(fs.exists(path1));
+      FileStatus after[] = fs.globStatus(pathglob);
+      assertTrue("trash dir before and after DROP TABLE noPURGE are not different",
+                 before.length != after.length);
+
+      // drop a table without saving to trash by setting the purge option
+      Table table2 = hm.getTable(dbName, ts.get(1));
+      assertNotNull(table2);
+      assertEquals(ts.get(1), table2.getTableName());
+      Path path2 = table2.getPath();
+      assertTrue(fs.exists(path2));
+      Path trash2 = mergePaths(trashDir, path2);
+      System.out.println("trashDir2 is " + trash2);
+      pathglob = trash2.suffix("*");
+      before = fs.globStatus(pathglob);
+      hm.dropTable(dbName, ts.get(1), true, true, true); // deleteData, ignoreUnknownTable, ifPurge
+      assertFalse(fs.exists(path2));
+      after = fs.globStatus(pathglob);
+      Arrays.sort(before);
+      Arrays.sort(after);
+      assertEquals("trash dir before and after DROP TABLE PURGE are different",
+                   before.length, after.length);
+      assertTrue("trash dir before and after DROP TABLE PURGE are different",
+                 Arrays.equals(before, after));
+
+      // Drop all tables
+      for (String tableName : hm.getAllTables(dbName)) {
+        Table table = hm.getTable(dbName, tableName);
+        hm.dropTable(dbName, tableName);
+        assertFalse(fs.exists(table.getPath()));
+      }
+      hm.dropDatabase(dbName);
+    } catch (Throwable e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testDropTableTrash() failed");
+      throw e;
+    }
+  }
+
+
   public void testPartition() throws Throwable {
     try {
       String tableName = "table_for_testpartition";
@@ -533,4 +620,39 @@ public class TestHive extends TestCase {
     newHiveObj = Hive.get(newHconf);
     assertTrue(prevHiveObj != newHiveObj);
   }
+
+  // shamelessly copied from Path in hadoop-2
+  private static final String SEPARATOR = "/";
+  private static final char SEPARATOR_CHAR = '/';
+
+  private static final String CUR_DIR = ".";
+
+  private static final boolean WINDOWS
+      = System.getProperty("os.name").startsWith("Windows");
+
+  private static final Pattern hasDriveLetterSpecifier =
+      Pattern.compile("^/?[a-zA-Z]:");
+
+  private static Path mergePaths(Path path1, Path path2) {
+    String path2Str = path2.toUri().getPath();
+    path2Str = path2Str.substring(startPositionWithoutWindowsDrive(path2Str));
+    // Add path components explicitly, because simply concatenating two path
+    // string is not safe, for example:
+    // "/" + "/foo" yields "//foo", which will be parsed as authority in Path
+    return new Path(path1.toUri().getScheme(),
+        path1.toUri().getAuthority(),
+        path1.toUri().getPath() + path2Str);
+  }
+
+  private static int startPositionWithoutWindowsDrive(String path) {
+    if (hasWindowsDrive(path)) {
+      return path.charAt(0) ==  SEPARATOR_CHAR ? 3 : 2;
+    } else {
+      return 0;
+    }
+  }
+
+  private static boolean hasWindowsDrive(String path) {
+    return (WINDOWS && hasDriveLetterSpecifier.matcher(path).find());
+  }
 }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveRemote.java Tue Oct 14 19:06:45 2014
@@ -84,6 +84,13 @@ public class TestHiveRemote extends Test
   }
 
   /**
+   * Cannot control trash in remote metastore, so skip this test
+   */
+  @Override
+  public void testDropTableTrash() {
+  }
+
+  /**
    * Finds a free port.
    *
    * @return a free port

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestIUD.java Tue Oct 14 19:06:45 2014
@@ -24,6 +24,9 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+/**
+ * various Parser tests for INSERT/UPDATE/DELETE
+ */
 public class TestIUD {
   private static HiveConf conf;
 
@@ -102,6 +105,18 @@ public class TestIUD {
       ast.toStringTree());
   }
   @Test
+  public void testUpdateWithWhereSingleSetExpr() throws ParseException {
+    ASTNode ast = parse("UPDATE src SET key = -3+(5*9)%8, val = cast(6.1 + c as INT), d = d - 1 WHERE value IS NULL");
+    Assert.assertEquals("AST doesn't match",
+      "(TOK_UPDATE_TABLE (TOK_TABNAME src) " +
+        "(TOK_SET_COLUMNS_CLAUSE " +
+        "(= (TOK_TABLE_OR_COL key) (+ (- 3) (% (* 5 9) 8))) " +
+        "(= (TOK_TABLE_OR_COL val) (TOK_FUNCTION TOK_INT (+ 6.1 (TOK_TABLE_OR_COL c)))) " +
+        "(= (TOK_TABLE_OR_COL d) (- (TOK_TABLE_OR_COL d) 1))) " +
+        "(TOK_WHERE (TOK_FUNCTION TOK_ISNULL (TOK_TABLE_OR_COL value))))",
+      ast.toStringTree());
+  }
+  @Test
   public void testUpdateWithWhereMultiSet() throws ParseException {
     ASTNode ast = parse("UPDATE src SET key = 3, value = 8 WHERE VALUE = 1230997");
     Assert.assertEquals("AST doesn't match", 
@@ -207,13 +222,13 @@ public class TestIUD {
   }
   @Test
   public void testInsertIntoTableFromAnonymousTable() throws ParseException {
-    ASTNode ast = parse("insert into table page_view values(1,2),(3,4)");
+    ASTNode ast = parse("insert into table page_view values(-1,2),(3,+4)");
     Assert.assertEquals("AST doesn't match",
       "(TOK_QUERY " +
         "(TOK_FROM " +
           "(TOK_VIRTUAL_TABLE " +
           "(TOK_VIRTUAL_TABREF TOK_ANONYMOUS) " +
-          "(TOK_VALUES_TABLE (TOK_VALUE_ROW 1 2) (TOK_VALUE_ROW 3 4)))) " +
+          "(TOK_VALUES_TABLE (TOK_VALUE_ROW (- 1) 2) (TOK_VALUE_ROW 3 (+ 4))))) " +
         "(TOK_INSERT (TOK_INSERT_INTO (TOK_TAB (TOK_TABNAME page_view))) " +
           "(TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))",
       ast.toStringTree());

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java Tue Oct 14 19:06:45 2014
@@ -198,7 +198,7 @@ public class TestUpdateDeleteSemanticAna
   @Test
   public void testInsertValues() throws Exception {
     try {
-      ReturnInfo rc = parseAndAnalyze("insert into table T values ('abc', 3), ('ghi', 5)",
+      ReturnInfo rc = parseAndAnalyze("insert into table T values ('abc', 3), ('ghi', null)",
           "testInsertValues");
 
       LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
@@ -266,9 +266,12 @@ public class TestUpdateDeleteSemanticAna
 
     // I have to create the tables here (rather than in setup()) because I need the Hive
     // connection, which is conviently created by the semantic analyzer.
-    db.createTable("T", Arrays.asList("a", "b"), null, OrcInputFormat.class, OrcOutputFormat.class);
+    Map<String, String> params = new HashMap<String, String>(1);
+    params.put(SemanticAnalyzer.ACID_TABLE_PROPERTY, "true");
+    db.createTable("T", Arrays.asList("a", "b"), null, OrcInputFormat.class,
+        OrcOutputFormat.class, 2, Arrays.asList("a"), params);
     db.createTable("U", Arrays.asList("a", "b"), Arrays.asList("ds"), OrcInputFormat.class,
-        OrcOutputFormat.class);
+        OrcOutputFormat.class, 2, Arrays.asList("a"), params);
     Table u = db.getTable("U");
     Map<String, String> partVals = new HashMap<String, String>(2);
     partVals.put("ds", "yesterday");
@@ -280,7 +283,7 @@ public class TestUpdateDeleteSemanticAna
     // validate the plan
     sem.validate();
 
-    QueryPlan plan = new QueryPlan(query, sem, 0L, testName);
+    QueryPlan plan = new QueryPlan(query, sem, 0L, testName, null);
 
     return new ReturnInfo(tree, sem, plan);
   }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/plan/TestTezWork.java Tue Oct 14 19:06:45 2014
@@ -23,11 +23,16 @@ import java.util.List;
 import junit.framework.Assert;
 
 import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
+import org.apache.hadoop.mapred.JobConf;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestTezWork {
 
+  private static final String MR_JAR_PROPERTY = "tmpjars";
   private List<BaseWork> nodes;
   private TezWork work;
 
@@ -156,4 +161,75 @@ public class TestTezWork {
       Assert.assertEquals(sorted.get(i), nodes.get(4-i));
     }
   }
+
+  @Test
+  public void testConfigureJars() throws Exception {
+    final JobConf conf = new JobConf();
+    conf.set(MR_JAR_PROPERTY, "file:///tmp/foo1.jar");
+    BaseWork baseWork = Mockito.mock(BaseWork.class);
+    Mockito.doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        conf.set(MR_JAR_PROPERTY, "file:///tmp/foo2.jar");
+        return null;
+      }
+
+    }).when(baseWork).configureJobConf(conf);
+
+    work.add(baseWork);
+    work.configureJobConfAndExtractJars(conf);
+    Assert.assertEquals("file:///tmp/foo1.jar,file:///tmp/foo2.jar", conf.get(MR_JAR_PROPERTY));
+  }
+
+  @Test
+  public void testConfigureJarsNoExtraJars() throws Exception {
+    final JobConf conf = new JobConf();
+    conf.set(MR_JAR_PROPERTY, "file:///tmp/foo1.jar");
+    BaseWork baseWork = Mockito.mock(BaseWork.class);
+
+    work.add(baseWork);
+    work.configureJobConfAndExtractJars(conf);
+    Assert.assertEquals("file:///tmp/foo1.jar", conf.get(MR_JAR_PROPERTY));
+  }
+
+  @Test
+  public void testConfigureJarsWithNull() throws Exception {
+    final JobConf conf = new JobConf();
+    conf.set(MR_JAR_PROPERTY, "file:///tmp/foo1.jar");
+    BaseWork baseWork = Mockito.mock(BaseWork.class);
+    Mockito.doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        conf.unset(MR_JAR_PROPERTY);
+        return null;
+      }
+
+    }).when(baseWork).configureJobConf(conf);
+
+    work.add(baseWork);
+    work.configureJobConfAndExtractJars(conf);
+    Assert.assertEquals("file:///tmp/foo1.jar", conf.get(MR_JAR_PROPERTY));
+  }
+
+  @Test
+  public void testConfigureJarsStartingWithNull() throws Exception {
+    final JobConf conf = new JobConf();
+    conf.unset(MR_JAR_PROPERTY);
+    BaseWork baseWork = Mockito.mock(BaseWork.class);
+    Mockito.doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        conf.setStrings(MR_JAR_PROPERTY, "file:///tmp/foo1.jar", "file:///tmp/foo2.jar");
+        return null;
+      }
+
+    }).when(baseWork).configureJobConf(conf);
+
+    work.add(baseWork);
+    work.configureJobConfAndExtractJars(conf);
+    Assert.assertEquals("file:///tmp/foo1.jar,file:///tmp/foo2.jar", conf.get(MR_JAR_PROPERTY));
+  }
 }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/TestSQLStdHiveAccessControllerCLI.java Tue Oct 14 19:06:45 2014
@@ -25,6 +25,8 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.DisallowTransformHook;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.Builder;
@@ -77,8 +79,9 @@ public class TestSQLStdHiveAccessControl
     HiveConf processedConf = new HiveConf();
     processedConf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
     try {
-      SQLStdHiveAccessController accessController = new SQLStdHiveAccessController(null,
-          processedConf, new HadoopDefaultAuthenticator(), getCLISessionCtx());
+      HiveAuthorizerFactory authorizerFactory = new SQLStdHiveAuthorizerFactory();
+      HiveAuthorizer authorizer = authorizerFactory.createHiveAuthorizer(null, processedConf,
+          new HadoopDefaultAuthenticator(), getCLISessionCtx());
       fail("Exception expected");
     } catch (HiveAuthzPluginException e) {
       assertTrue(e.getMessage().contains(

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java Tue Oct 14 19:06:45 2014
@@ -63,12 +63,13 @@ public abstract class CompactorTest {
   protected CompactionTxnHandler txnHandler;
   protected IMetaStoreClient ms;
   protected long sleepTime = 1000;
+  protected HiveConf conf;
 
   private final MetaStoreThread.BooleanPointer stop = new MetaStoreThread.BooleanPointer();
   private final File tmpdir;
 
   protected CompactorTest() throws Exception {
-    HiveConf conf = new HiveConf();
+    conf = new HiveConf();
     TxnDbUtil.setConfValues(conf);
     TxnDbUtil.cleanDb();
     ms = new HiveMetaStoreClient(conf);
@@ -79,16 +80,20 @@ public abstract class CompactorTest {
     tmpdir.deleteOnExit();
   }
 
-  protected void startInitiator(HiveConf conf) throws Exception {
-    startThread('i', conf);
+  protected void startInitiator() throws Exception {
+    startThread('i', true);
   }
 
-  protected void startWorker(HiveConf conf) throws Exception {
-    startThread('w', conf);
+  protected void startWorker() throws Exception {
+    startThread('w', true);
   }
 
-  protected void startCleaner(HiveConf conf) throws Exception {
-    startThread('c', conf);
+  protected void startCleaner() throws Exception {
+    startThread('c', true);
+  }
+
+  protected void startCleaner(MetaStoreThread.BooleanPointer looped) throws Exception {
+    startThread('c', false, looped);
   }
 
   protected Table newTable(String dbName, String tableName, boolean partitioned) throws TException {
@@ -117,6 +122,9 @@ public abstract class CompactorTest {
 
     table.setParameters(parameters);
 
+    // drop the table first, in case some previous test created it
+    ms.dropTable(dbName, tableName);
+
     ms.createTable(table);
     return table;
   }
@@ -142,37 +150,27 @@ public abstract class CompactorTest {
     return txns.get(0);
   }
 
-  protected void addDeltaFile(HiveConf conf, Table t, Partition p, long minTxn, long maxTxn,
-                              int numRecords) throws Exception{
-    addFile(conf, t, p, minTxn, maxTxn, numRecords, FileType.DELTA, 2, true);
-  }
-
-  protected void addBaseFile(HiveConf conf, Table t, Partition p, long maxTxn,
-                             int numRecords) throws Exception{
-    addFile(conf, t, p, 0, maxTxn, numRecords, FileType.BASE, 2, true);
+  protected void addDeltaFile(Table t, Partition p, long minTxn, long maxTxn, int numRecords)
+      throws Exception {
+    addFile(t, p, minTxn, maxTxn, numRecords, FileType.DELTA, 2, true);
   }
 
-  protected void addLegacyFile(HiveConf conf, Table t, Partition p,
-                               int numRecords) throws Exception {
-    addFile(conf, t, p, 0, 0, numRecords, FileType.LEGACY, 2, true);
+  protected void addBaseFile(Table t, Partition p, long maxTxn, int numRecords) throws Exception {
+    addFile(t, p, 0, maxTxn, numRecords, FileType.BASE, 2, true);
   }
 
-  protected void addDeltaFile(HiveConf conf, Table t, Partition p, long minTxn, long maxTxn,
-                              int numRecords, int numBuckets, boolean allBucketsPresent)
-      throws Exception {
-    addFile(conf, t, p, minTxn, maxTxn, numRecords, FileType.DELTA, numBuckets, allBucketsPresent);
+  protected void addLegacyFile(Table t, Partition p, int numRecords) throws Exception {
+    addFile(t, p, 0, 0, numRecords, FileType.LEGACY, 2, true);
   }
 
-  protected void addBaseFile(HiveConf conf, Table t, Partition p, long maxTxn,
-                             int numRecords, int numBuckets, boolean allBucketsPresent)
-      throws Exception {
-    addFile(conf, t, p, 0, maxTxn, numRecords, FileType.BASE, numBuckets, allBucketsPresent);
+  protected void addDeltaFile(Table t, Partition p, long minTxn, long maxTxn, int numRecords,
+                              int numBuckets, boolean allBucketsPresent) throws Exception {
+    addFile(t, p, minTxn, maxTxn, numRecords, FileType.DELTA, numBuckets, allBucketsPresent);
   }
 
-  protected void addLegacyFile(HiveConf conf, Table t, Partition p,
-                               int numRecords, int numBuckets, boolean allBucketsPresent)
-      throws Exception {
-    addFile(conf, t, p, 0, 0, numRecords, FileType.LEGACY, numBuckets, allBucketsPresent);
+  protected void addBaseFile(Table t, Partition p, long maxTxn, int numRecords, int numBuckets,
+                             boolean allBucketsPresent) throws Exception {
+    addFile(t, p, 0, maxTxn, numRecords, FileType.BASE, numBuckets, allBucketsPresent);
   }
 
   protected List<Path> getDirectories(HiveConf conf, Table t, Partition p) throws Exception {
@@ -191,6 +189,10 @@ public abstract class CompactorTest {
     for (long tid : rsp.getTxn_ids()) txnHandler.commitTxn(new CommitTxnRequest(tid));
   }
 
+  protected void stopThread() {
+    stop.boolVal = true;
+  }
+
   private StorageDescriptor newStorageDescriptor(String location, List<Order> sortCols) {
     StorageDescriptor sd = new StorageDescriptor();
     List<FieldSchema> cols = new ArrayList<FieldSchema>(2);
@@ -214,9 +216,13 @@ public abstract class CompactorTest {
     return sd;
   }
 
-  // I can't do this with @Before because I want to be able to control the config file provided
-  // to each test.
-  private void startThread(char type, HiveConf conf) throws Exception {
+  // I can't do this with @Before because I want to be able to control when the thead starts
+  private void startThread(char type, boolean stopAfterOne) throws Exception {
+    startThread(type, stopAfterOne, new MetaStoreThread.BooleanPointer());
+  }
+
+  private void startThread(char type, boolean stopAfterOne, MetaStoreThread.BooleanPointer looped)
+    throws Exception {
     TxnDbUtil.setConfValues(conf);
     CompactorThread t = null;
     switch (type) {
@@ -227,9 +233,10 @@ public abstract class CompactorTest {
     }
     t.setThreadId((int) t.getId());
     t.setHiveConf(conf);
-    stop.boolVal = true;
-    t.init(stop);
-    t.run();
+    stop.boolVal = stopAfterOne;
+    t.init(stop, looped);
+    if (stopAfterOne) t.run();
+    else t.start();
   }
 
   private String getLocation(String tableName, String partValue) {
@@ -243,7 +250,7 @@ public abstract class CompactorTest {
 
   private enum FileType {BASE, DELTA, LEGACY};
 
-  private void addFile(HiveConf conf, Table t, Partition p, long minTxn, long maxTxn,
+  private void addFile(Table t, Partition p, long minTxn, long maxTxn,
                        int numRecords,  FileType type, int numBuckets,
                        boolean allBucketsPresent) throws Exception {
     String partValue = (p == null) ? null : p.getValues().get(0);
@@ -332,6 +339,7 @@ public abstract class CompactorTest {
     private final Configuration conf;
     private FSDataInputStream is = null;
     private final FileSystem fs;
+    private boolean lastWasDelete = true;
 
     MockRawReader(Configuration conf, List<Path> files) throws IOException {
       filesToRead = new Stack<Path>();
@@ -346,6 +354,15 @@ public abstract class CompactorTest {
     }
 
     @Override
+    public boolean isDelete(Text value) {
+      // Alternate between returning deleted and not.  This is easier than actually
+      // tracking operations. We test that this is getting properly called by checking that only
+      // half the records show up in base files after major compactions.
+      lastWasDelete = !lastWasDelete;
+      return lastWasDelete;
+    }
+
+    @Override
     public boolean next(RecordIdentifier identifier, Text text) throws IOException {
       if (is == null) {
         // Open the next file

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java Tue Oct 14 19:06:45 2014
@@ -18,21 +18,26 @@
 package org.apache.hadoop.hive.ql.txn.compactor;
 
 import junit.framework.Assert;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStoreThread;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
-import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
-import org.junit.Before;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Tests for the compactor Cleaner thread
  */
 public class TestCleaner extends CompactorTest {
+
+  static final private Log LOG = LogFactory.getLog(TestCleaner.class.getName());
+
   public TestCleaner() throws Exception {
     super();
   }
@@ -41,19 +46,17 @@ public class TestCleaner extends Compact
   public void nothing() throws Exception {
     // Test that the whole things works when there's nothing in the queue.  This is just a
     // survival test.
-    startCleaner(new HiveConf());
+    startCleaner();
   }
 
   @Test
   public void cleanupAfterMajorTableCompaction() throws Exception {
     Table t = newTable("default", "camtc", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
-    addBaseFile(conf, t, null, 25L, 25);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
+    addBaseFile(t, null, 25L, 25);
 
     burnThroughTransactions(25);
 
@@ -63,7 +66,7 @@ public class TestCleaner extends Compact
     txnHandler.markCompacted(ci);
     txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -80,12 +83,10 @@ public class TestCleaner extends Compact
     Table t = newTable("default", "campc", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
-    addBaseFile(conf, t, p, 25L, 25);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
 
     burnThroughTransactions(25);
 
@@ -96,7 +97,7 @@ public class TestCleaner extends Compact
     txnHandler.markCompacted(ci);
     txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -112,12 +113,10 @@ public class TestCleaner extends Compact
   public void cleanupAfterMinorTableCompaction() throws Exception {
     Table t = newTable("default", "camitc", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
-    addDeltaFile(conf, t, null, 21L, 24L, 4);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
+    addDeltaFile(t, null, 21L, 24L, 4);
 
     burnThroughTransactions(25);
 
@@ -127,7 +126,7 @@ public class TestCleaner extends Compact
     txnHandler.markCompacted(ci);
     txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -151,12 +150,10 @@ public class TestCleaner extends Compact
     Table t = newTable("default", "camipc", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
-    addDeltaFile(conf, t, p, 21L, 24L, 4);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addDeltaFile(t, p, 21L, 24L, 4);
 
     burnThroughTransactions(25);
 
@@ -167,7 +164,7 @@ public class TestCleaner extends Compact
     txnHandler.markCompacted(ci);
     txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -190,12 +187,10 @@ public class TestCleaner extends Compact
   public void blockedByLockTable() throws Exception {
     Table t = newTable("default", "bblt", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
-    addDeltaFile(conf, t, null, 21L, 24L, 4);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
+    addDeltaFile(t, null, 21L, 24L, 4);
 
     burnThroughTransactions(25);
 
@@ -212,7 +207,7 @@ public class TestCleaner extends Compact
     LockRequest req = new LockRequest(components, "me", "localhost");
     LockResponse res = txnHandler.lock(req);
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -228,12 +223,10 @@ public class TestCleaner extends Compact
     Table t = newTable("default", "bblp", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
-    addDeltaFile(conf, t, p, 21L, 24L, 4);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addDeltaFile(t, p, 21L, 24L, 4);
 
     burnThroughTransactions(25);
 
@@ -244,7 +237,7 @@ public class TestCleaner extends Compact
     txnHandler.markCompacted(ci);
     txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
 
-    LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
+    LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("bblp");
     comp.setPartitionname("ds=today");
     List<LockComponent> components = new ArrayList<LockComponent>(1);
@@ -252,7 +245,7 @@ public class TestCleaner extends Compact
     LockRequest req = new LockRequest(components, "me", "localhost");
     LockResponse res = txnHandler.lock(req);
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -265,15 +258,154 @@ public class TestCleaner extends Compact
   }
 
   @Test
+  public void notBlockedBySubsequentLock() throws Exception {
+    Table t = newTable("default", "bblt", false);
+
+    // Set the run frequency low on this test so it doesn't take long
+    conf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, 100,
+        TimeUnit.MILLISECONDS);
+
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
+    addDeltaFile(t, null, 21L, 24L, 4);
+
+    burnThroughTransactions(25);
+
+    CompactionRequest rqst = new CompactionRequest("default", "bblt", CompactionType.MINOR);
+    txnHandler.compact(rqst);
+    CompactionInfo ci = txnHandler.findNextToCompact("fred");
+    txnHandler.markCompacted(ci);
+    txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
+
+    LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default");
+    comp.setTablename("bblt");
+    List<LockComponent> components = new ArrayList<LockComponent>(1);
+    components.add(comp);
+    LockRequest req = new LockRequest(components, "me", "localhost");
+    LockResponse res = txnHandler.lock(req);
+
+    MetaStoreThread.BooleanPointer looped = new MetaStoreThread.BooleanPointer();
+    looped.boolVal = false;
+    startCleaner(looped);
+
+    // Make sure the compactor has a chance to run once
+    while (!looped.boolVal) {
+      Thread.currentThread().sleep(100);
+    }
+
+    // There should still be one request, as the locks still held.
+    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
+    List<ShowCompactResponseElement> compacts = rsp.getCompacts();
+    Assert.assertEquals(1, compacts.size());
+
+    // obtain a second lock.  This shouldn't block cleaner as it was acquired after the initial
+    // clean request
+    LockComponent comp2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default");
+    comp2.setTablename("bblt");
+    List<LockComponent> components2 = new ArrayList<LockComponent>(1);
+    components2.add(comp2);
+    LockRequest req2 = new LockRequest(components, "me", "localhost");
+    LockResponse res2 = txnHandler.lock(req2);
+
+    // Unlock the previous lock
+    txnHandler.unlock(new UnlockRequest(res.getLockid()));
+    looped.boolVal = false;
+
+    while (!looped.boolVal) {
+      Thread.currentThread().sleep(100);
+    }
+    stopThread();
+    Thread.currentThread().sleep(200);
+
+
+    // Check there are no compactions requests left.
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    compacts = rsp.getCompacts();
+    Assert.assertEquals(0, compacts.size());
+  }
+
+  @Test
+  public void partitionNotBlockedBySubsequentLock() throws Exception {
+    Table t = newTable("default", "bblt", true);
+    Partition p = newPartition(t, "today");
+
+    // Set the run frequency low on this test so it doesn't take long
+    conf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, 100,
+        TimeUnit.MILLISECONDS);
+
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addDeltaFile(t, p, 21L, 24L, 4);
+
+    burnThroughTransactions(25);
+
+    CompactionRequest rqst = new CompactionRequest("default", "bblt", CompactionType.MINOR);
+    rqst.setPartitionname("ds=today");
+    txnHandler.compact(rqst);
+    CompactionInfo ci = txnHandler.findNextToCompact("fred");
+    txnHandler.markCompacted(ci);
+    txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
+
+    LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "default");
+    comp.setTablename("bblt");
+    comp.setPartitionname("ds=today");
+    List<LockComponent> components = new ArrayList<LockComponent>(1);
+    components.add(comp);
+    LockRequest req = new LockRequest(components, "me", "localhost");
+    LockResponse res = txnHandler.lock(req);
+
+    MetaStoreThread.BooleanPointer looped = new MetaStoreThread.BooleanPointer();
+    looped.boolVal = false;
+    startCleaner(looped);
+
+    // Make sure the compactor has a chance to run once
+    while (!looped.boolVal) {
+      Thread.currentThread().sleep(100);
+    }
+
+    // There should still be one request, as the locks still held.
+    ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
+    List<ShowCompactResponseElement> compacts = rsp.getCompacts();
+    Assert.assertEquals(1, compacts.size());
+
+
+    // obtain a second lock.  This shouldn't block cleaner as it was acquired after the initial
+    // clean request
+    LockComponent comp2 = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "default");
+    comp2.setTablename("bblt");
+    comp2.setPartitionname("ds=today");
+    List<LockComponent> components2 = new ArrayList<LockComponent>(1);
+    components2.add(comp2);
+    LockRequest req2 = new LockRequest(components, "me", "localhost");
+    LockResponse res2 = txnHandler.lock(req2);
+
+    // Unlock the previous lock
+    txnHandler.unlock(new UnlockRequest(res.getLockid()));
+    looped.boolVal = false;
+
+    while (!looped.boolVal) {
+      Thread.currentThread().sleep(100);
+    }
+    stopThread();
+    Thread.currentThread().sleep(200);
+
+
+    // Check there are no compactions requests left.
+    rsp = txnHandler.showCompact(new ShowCompactRequest());
+    compacts = rsp.getCompacts();
+    Assert.assertEquals(0, compacts.size());
+  }
+
+  @Test
   public void cleanupAfterMajorPartitionCompactionNoBase() throws Exception {
     Table t = newTable("default", "campcnb", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addDeltaFile(conf, t, p, 1L, 22L, 22);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
-    addBaseFile(conf, t, p, 25L, 25);
+    addDeltaFile(t, p, 1L, 22L, 22);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 25L, 25);
 
     burnThroughTransactions(25);
 
@@ -284,7 +416,7 @@ public class TestCleaner extends Compact
     txnHandler.markCompacted(ci);
     txnHandler.setRunAs(ci.id, System.getProperty("user.name"));
 
-    startCleaner(conf);
+    startCleaner();
 
     // Check there are no compactions requests left.
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
@@ -295,9 +427,4 @@ public class TestCleaner extends Compact
     Assert.assertEquals(1, paths.size());
     Assert.assertEquals("base_25", paths.get(0).getName());
   }
-
-  @Before
-  public void setUpTxnDb() throws Exception {
-    TxnDbUtil.setConfValues(new HiveConf());
-  }
 }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java Tue Oct 14 19:06:45 2014
@@ -47,7 +47,7 @@ public class TestInitiator extends Compa
   public void nothing() throws Exception {
     // Test that the whole things works when there's nothing in the queue.  This is just a
     // survival test.
-    startInitiator(new HiveConf());
+    startInitiator();
   }
 
   @Test
@@ -63,7 +63,7 @@ public class TestInitiator extends Compa
     txnHandler.findNextToCompact(Worker.hostname() + "-193892");
     txnHandler.findNextToCompact("nosuchhost-193892");
 
-    startInitiator(new HiveConf());
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -89,10 +89,9 @@ public class TestInitiator extends Compa
 
     txnHandler.findNextToCompact("nosuchhost-193892");
 
-    HiveConf conf = new HiveConf();
     conf.setTimeVar(HiveConf.ConfVars.HIVE_COMPACTOR_WORKER_TIMEOUT, 1L, TimeUnit.MILLISECONDS);
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -104,7 +103,6 @@ public class TestInitiator extends Compa
   public void majorCompactOnTableTooManyAborts() throws Exception {
     Table t = newTable("default", "mcottma", false);
 
-    HiveConf conf = new HiveConf();
     HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 10);
 
     for (int i = 0; i < 11; i++) {
@@ -119,7 +117,7 @@ public class TestInitiator extends Compa
       txnHandler.abortTxn(new AbortTxnRequest(txnid));
     }
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -134,7 +132,6 @@ public class TestInitiator extends Compa
     Table t = newTable("default", "mcoptma", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
     HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 10);
 
     for (int i = 0; i < 11; i++) {
@@ -150,7 +147,7 @@ public class TestInitiator extends Compa
       txnHandler.abortTxn(new AbortTxnRequest(txnid));
     }
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -168,7 +165,6 @@ public class TestInitiator extends Compa
       Partition p = newPartition(t, "day-" + i);
     }
 
-    HiveConf conf = new HiveConf();
     HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 10);
 
     for (int i = 0; i < 11; i++) {
@@ -184,7 +180,7 @@ public class TestInitiator extends Compa
       txnHandler.abortTxn(new AbortTxnRequest(txnid));
     }
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     Assert.assertEquals(0, rsp.getCompactsSize());
@@ -197,8 +193,6 @@ public class TestInitiator extends Compa
     // accidently clean it too.
     Table t = newTable("default", "ceat", false);
 
-    HiveConf conf = new HiveConf();
-
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("ceat");
@@ -216,7 +210,7 @@ public class TestInitiator extends Compa
     GetOpenTxnsResponse openTxns = txnHandler.getOpenTxns();
     Assert.assertEquals(101, openTxns.getOpen_txnsSize());
 
-    startInitiator(conf);
+    startInitiator();
 
     openTxns = txnHandler.getOpenTxns();
     Assert.assertEquals(1, openTxns.getOpen_txnsSize());
@@ -228,7 +222,6 @@ public class TestInitiator extends Compa
     parameters.put("NO_AUTO_COMPACTION", "true");
     Table t = newTable("default", "ncwncs", false, parameters);
 
-    HiveConf conf = new HiveConf();
     HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 10);
 
     for (int i = 0; i < 11; i++) {
@@ -243,7 +236,7 @@ public class TestInitiator extends Compa
       txnHandler.abortTxn(new AbortTxnRequest(txnid));
     }
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     Assert.assertEquals(0, rsp.getCompactsSize());
@@ -253,7 +246,6 @@ public class TestInitiator extends Compa
   public void noCompactWhenCompactAlreadyScheduled() throws Exception {
     Table t = newTable("default", "ncwcas", false);
 
-    HiveConf conf = new HiveConf();
     HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 10);
 
     for (int i = 0; i < 11; i++) {
@@ -277,7 +269,7 @@ public class TestInitiator extends Compa
     Assert.assertEquals("initiated", compacts.get(0).getState());
     Assert.assertEquals("ncwcas", compacts.get(0).getTablename());
 
-    startInitiator(conf);
+    startInitiator();
 
     rsp = txnHandler.showCompact(new ShowCompactRequest());
     compacts = rsp.getCompacts();
@@ -291,11 +283,9 @@ public class TestInitiator extends Compa
   public void compactTableHighDeltaPct() throws Exception {
     Table t = newTable("default", "cthdp", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
 
     burnThroughTransactions(23);
 
@@ -309,7 +299,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -324,11 +314,9 @@ public class TestInitiator extends Compa
     Table t = newTable("default", "cphdp", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
 
     burnThroughTransactions(23);
 
@@ -343,7 +331,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -358,11 +346,9 @@ public class TestInitiator extends Compa
   public void noCompactTableDeltaPctNotHighEnough() throws Exception {
     Table t = newTable("default", "nctdpnhe", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 50L, 50);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
+    addBaseFile(t, null, 50L, 50);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
 
     burnThroughTransactions(53);
 
@@ -376,7 +362,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     Assert.assertEquals(0, rsp.getCompactsSize());
@@ -386,20 +372,18 @@ public class TestInitiator extends Compa
   public void compactTableTooManyDeltas() throws Exception {
     Table t = newTable("default", "cttmd", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 200L, 200);
-    addDeltaFile(conf, t, null, 201L, 201L, 1);
-    addDeltaFile(conf, t, null, 202L, 202L, 1);
-    addDeltaFile(conf, t, null, 203L, 203L, 1);
-    addDeltaFile(conf, t, null, 204L, 204L, 1);
-    addDeltaFile(conf, t, null, 205L, 205L, 1);
-    addDeltaFile(conf, t, null, 206L, 206L, 1);
-    addDeltaFile(conf, t, null, 207L, 207L, 1);
-    addDeltaFile(conf, t, null, 208L, 208L, 1);
-    addDeltaFile(conf, t, null, 209L, 209L, 1);
-    addDeltaFile(conf, t, null, 210L, 210L, 1);
-    addDeltaFile(conf, t, null, 211L, 211L, 1);
+    addBaseFile(t, null, 200L, 200);
+    addDeltaFile(t, null, 201L, 201L, 1);
+    addDeltaFile(t, null, 202L, 202L, 1);
+    addDeltaFile(t, null, 203L, 203L, 1);
+    addDeltaFile(t, null, 204L, 204L, 1);
+    addDeltaFile(t, null, 205L, 205L, 1);
+    addDeltaFile(t, null, 206L, 206L, 1);
+    addDeltaFile(t, null, 207L, 207L, 1);
+    addDeltaFile(t, null, 208L, 208L, 1);
+    addDeltaFile(t, null, 209L, 209L, 1);
+    addDeltaFile(t, null, 210L, 210L, 1);
+    addDeltaFile(t, null, 211L, 211L, 1);
 
     burnThroughTransactions(210);
 
@@ -413,7 +397,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -428,20 +412,18 @@ public class TestInitiator extends Compa
     Table t = newTable("default", "cptmd", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, p, 200L, 200);
-    addDeltaFile(conf, t, p, 201L, 201L, 1);
-    addDeltaFile(conf, t, p, 202L, 202L, 1);
-    addDeltaFile(conf, t, p, 203L, 203L, 1);
-    addDeltaFile(conf, t, p, 204L, 204L, 1);
-    addDeltaFile(conf, t, p, 205L, 205L, 1);
-    addDeltaFile(conf, t, p, 206L, 206L, 1);
-    addDeltaFile(conf, t, p, 207L, 207L, 1);
-    addDeltaFile(conf, t, p, 208L, 208L, 1);
-    addDeltaFile(conf, t, p, 209L, 209L, 1);
-    addDeltaFile(conf, t, p, 210L, 210L, 1);
-    addDeltaFile(conf, t, p, 211L, 211L, 1);
+    addBaseFile(t, p, 200L, 200);
+    addDeltaFile(t, p, 201L, 201L, 1);
+    addDeltaFile(t, p, 202L, 202L, 1);
+    addDeltaFile(t, p, 203L, 203L, 1);
+    addDeltaFile(t, p, 204L, 204L, 1);
+    addDeltaFile(t, p, 205L, 205L, 1);
+    addDeltaFile(t, p, 206L, 206L, 1);
+    addDeltaFile(t, p, 207L, 207L, 1);
+    addDeltaFile(t, p, 208L, 208L, 1);
+    addDeltaFile(t, p, 209L, 209L, 1);
+    addDeltaFile(t, p, 210L, 210L, 1);
+    addDeltaFile(t, p, 211L, 211L, 1);
 
     burnThroughTransactions(210);
 
@@ -456,7 +438,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -471,11 +453,9 @@ public class TestInitiator extends Compa
   public void noCompactTableNotEnoughDeltas() throws Exception {
     Table t = newTable("default", "nctned", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 200L, 200);
-    addDeltaFile(conf, t, null, 201L, 205L, 5);
-    addDeltaFile(conf, t, null, 206L, 211L, 6);
+    addBaseFile(t, null, 200L, 200);
+    addDeltaFile(t, null, 201L, 205L, 5);
+    addDeltaFile(t, null, 206L, 211L, 6);
 
     burnThroughTransactions(210);
 
@@ -489,7 +469,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     Assert.assertEquals(0, rsp.getCompactsSize());
@@ -499,20 +479,18 @@ public class TestInitiator extends Compa
   public void chooseMajorOverMinorWhenBothValid() throws Exception {
     Table t = newTable("default", "cmomwbv", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 200L, 200);
-    addDeltaFile(conf, t, null, 201L, 211L, 11);
-    addDeltaFile(conf, t, null, 212L, 222L, 11);
-    addDeltaFile(conf, t, null, 223L, 233L, 11);
-    addDeltaFile(conf, t, null, 234L, 244L, 11);
-    addDeltaFile(conf, t, null, 245L, 255L, 11);
-    addDeltaFile(conf, t, null, 256L, 266L, 11);
-    addDeltaFile(conf, t, null, 267L, 277L, 11);
-    addDeltaFile(conf, t, null, 278L, 288L, 11);
-    addDeltaFile(conf, t, null, 289L, 299L, 11);
-    addDeltaFile(conf, t, null, 300L, 310L, 11);
-    addDeltaFile(conf, t, null, 311L, 321L, 11);
+    addBaseFile(t, null, 200L, 200);
+    addDeltaFile(t, null, 201L, 211L, 11);
+    addDeltaFile(t, null, 212L, 222L, 11);
+    addDeltaFile(t, null, 223L, 233L, 11);
+    addDeltaFile(t, null, 234L, 244L, 11);
+    addDeltaFile(t, null, 245L, 255L, 11);
+    addDeltaFile(t, null, 256L, 266L, 11);
+    addDeltaFile(t, null, 267L, 277L, 11);
+    addDeltaFile(t, null, 278L, 288L, 11);
+    addDeltaFile(t, null, 289L, 299L, 11);
+    addDeltaFile(t, null, 300L, 310L, 11);
+    addDeltaFile(t, null, 311L, 321L, 11);
 
     burnThroughTransactions(320);
 
@@ -526,7 +504,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -541,19 +519,17 @@ public class TestInitiator extends Compa
     Table t = newTable("default", "ednb", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addDeltaFile(conf, t, p, 1L, 201L, 200);
-    addDeltaFile(conf, t, p, 202L, 202L, 1);
-    addDeltaFile(conf, t, p, 203L, 203L, 1);
-    addDeltaFile(conf, t, p, 204L, 204L, 1);
-    addDeltaFile(conf, t, p, 205L, 205L, 1);
-    addDeltaFile(conf, t, p, 206L, 206L, 1);
-    addDeltaFile(conf, t, p, 207L, 207L, 1);
-    addDeltaFile(conf, t, p, 208L, 208L, 1);
-    addDeltaFile(conf, t, p, 209L, 209L, 1);
-    addDeltaFile(conf, t, p, 210L, 210L, 1);
-    addDeltaFile(conf, t, p, 211L, 211L, 1);
+    addDeltaFile(t, p, 1L, 201L, 200);
+    addDeltaFile(t, p, 202L, 202L, 1);
+    addDeltaFile(t, p, 203L, 203L, 1);
+    addDeltaFile(t, p, 204L, 204L, 1);
+    addDeltaFile(t, p, 205L, 205L, 1);
+    addDeltaFile(t, p, 206L, 206L, 1);
+    addDeltaFile(t, p, 207L, 207L, 1);
+    addDeltaFile(t, p, 208L, 208L, 1);
+    addDeltaFile(t, p, 209L, 209L, 1);
+    addDeltaFile(t, p, 210L, 210L, 1);
+    addDeltaFile(t, p, 211L, 211L, 1);
 
     burnThroughTransactions(210);
 
@@ -568,7 +544,7 @@ public class TestInitiator extends Compa
     LockResponse res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -584,11 +560,9 @@ public class TestInitiator extends Compa
     Table t = newTable("default", "ttospgocr", true);
     Partition p = newPartition(t, "today");
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
 
     burnThroughTransactions(23);
 
@@ -614,7 +588,7 @@ public class TestInitiator extends Compa
     res = txnHandler.lock(req);
     txnHandler.commitTxn(new CommitTxnRequest(txnid));
 
-    startInitiator(conf);
+    startInitiator();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -626,9 +600,4 @@ public class TestInitiator extends Compa
   }
 
   // TODO test compactions with legacy file types
-
-  @Before
-  public void setUpTxnDb() throws Exception {
-    TxnDbUtil.setConfValues(new HiveConf());
-  }
 }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java Tue Oct 14 19:06:45 2014
@@ -48,7 +48,7 @@ public class TestWorker extends Compacto
   public void nothing() throws Exception {
     // Test that the whole things works when there's nothing in the queue.  This is just a
     // survival test.
-    startWorker(new HiveConf());
+    startWorker();
   }
 
   @Test
@@ -205,19 +205,17 @@ public class TestWorker extends Compacto
 
     Table t = newTable("default", "st", false, new HashMap<String, String>(), sortCols);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
-    addDeltaFile(conf, t, null, 21L, 24L, 4);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
+    addDeltaFile(t, null, 21L, 24L, 4);
 
     burnThroughTransactions(25);
 
     CompactionRequest rqst = new CompactionRequest("default", "st", CompactionType.MINOR);
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     // There should still be four directories in the location.
     FileSystem fs = FileSystem.get(conf);
@@ -232,12 +230,11 @@ public class TestWorker extends Compacto
 
     Table t = newTable("default", "sp", true, new HashMap<String, String>(), sortCols);
     Partition p = newPartition(t, "today", sortCols);
-    HiveConf conf = new HiveConf();
 
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
-    addDeltaFile(conf, t, p, 21L, 24L, 4);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
+    addDeltaFile(t, p, 21L, 24L, 4);
 
     burnThroughTransactions(25);
 
@@ -245,7 +242,7 @@ public class TestWorker extends Compacto
     rqst.setPartitionname("ds=today");
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     // There should still be four directories in the location.
     FileSystem fs = FileSystem.get(conf);
@@ -258,18 +255,16 @@ public class TestWorker extends Compacto
     LOG.debug("Starting minorTableWithBase");
     Table t = newTable("default", "mtwb", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
 
     burnThroughTransactions(25);
 
     CompactionRequest rqst = new CompactionRequest("default", "mtwb", CompactionType.MINOR);
     txnHandler.compact(rqst);
 
-    startWorker(conf);
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -304,11 +299,10 @@ public class TestWorker extends Compacto
   public void minorPartitionWithBase() throws Exception {
     Table t = newTable("default", "mpwb", true);
     Partition p = newPartition(t, "today");
-    HiveConf conf = new HiveConf();
 
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
 
     burnThroughTransactions(25);
 
@@ -316,7 +310,7 @@ public class TestWorker extends Compacto
     rqst.setPartitionname("ds=today");
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -351,17 +345,15 @@ public class TestWorker extends Compacto
     LOG.debug("Starting minorTableWithBase");
     Table t = newTable("default", "mtnb", false);
 
-    HiveConf conf = new HiveConf();
-
-    addDeltaFile(conf, t, null, 1L, 2L, 2);
-    addDeltaFile(conf, t, null, 3L, 4L, 2);
+    addDeltaFile(t, null, 1L, 2L, 2);
+    addDeltaFile(t, null, 3L, 4L, 2);
 
     burnThroughTransactions(5);
 
     CompactionRequest rqst = new CompactionRequest("default", "mtnb", CompactionType.MINOR);
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -396,18 +388,16 @@ public class TestWorker extends Compacto
     LOG.debug("Starting majorTableWithBase");
     Table t = newTable("default", "matwb", false);
 
-    HiveConf conf = new HiveConf();
-
-    addBaseFile(conf, t, null, 20L, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
+    addBaseFile(t, null, 20L, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
 
     burnThroughTransactions(25);
 
     CompactionRequest rqst = new CompactionRequest("default", "matwb", CompactionType.MAJOR);
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -428,8 +418,8 @@ public class TestWorker extends Compacto
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
-        Assert.assertEquals(1248L, buckets[0].getLen());
-        Assert.assertEquals(1248L, buckets[1].getLen());
+        Assert.assertEquals(624L, buckets[0].getLen());
+        Assert.assertEquals(624L, buckets[1].getLen());
       } else {
         LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName());
       }
@@ -442,11 +432,10 @@ public class TestWorker extends Compacto
     LOG.debug("Starting majorPartitionWithBase");
     Table t = newTable("default", "mapwb", true);
     Partition p = newPartition(t, "today");
-    HiveConf conf = new HiveConf();
 
-    addBaseFile(conf, t, p, 20L, 20);
-    addDeltaFile(conf, t, p, 21L, 22L, 2);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
+    addBaseFile(t, p, 20L, 20);
+    addDeltaFile(t, p, 21L, 22L, 2);
+    addDeltaFile(t, p, 23L, 24L, 2);
 
     burnThroughTransactions(25);
 
@@ -454,7 +443,7 @@ public class TestWorker extends Compacto
     rqst.setPartitionname("ds=today");
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -475,8 +464,8 @@ public class TestWorker extends Compacto
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
-        Assert.assertEquals(1248L, buckets[0].getLen());
-        Assert.assertEquals(1248L, buckets[1].getLen());
+        Assert.assertEquals(624L, buckets[0].getLen());
+        Assert.assertEquals(624L, buckets[1].getLen());
       } else {
         LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName());
       }
@@ -489,17 +478,15 @@ public class TestWorker extends Compacto
     LOG.debug("Starting majorTableNoBase");
     Table t = newTable("default", "matnb", false);
 
-    HiveConf conf = new HiveConf();
-
-    addDeltaFile(conf, t, null, 1L, 2L, 2);
-    addDeltaFile(conf, t, null, 3L, 4L, 2);
+    addDeltaFile(t, null, 1L, 2L, 2);
+    addDeltaFile(t, null, 3L, 4L, 2);
 
     burnThroughTransactions(5);
 
     CompactionRequest rqst = new CompactionRequest("default", "matnb", CompactionType.MAJOR);
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -520,8 +507,8 @@ public class TestWorker extends Compacto
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
-        Assert.assertEquals(208L, buckets[0].getLen());
-        Assert.assertEquals(208L, buckets[1].getLen());
+        Assert.assertEquals(104L, buckets[0].getLen());
+        Assert.assertEquals(104L, buckets[1].getLen());
       } else {
         LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName());
       }
@@ -534,18 +521,16 @@ public class TestWorker extends Compacto
     LOG.debug("Starting majorTableLegacy");
     Table t = newTable("default", "matl", false);
 
-    HiveConf conf = new HiveConf();
-
-    addLegacyFile(conf, t, null, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
+    addLegacyFile(t, null, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
 
     burnThroughTransactions(25);
 
     CompactionRequest rqst = new CompactionRequest("default", "matl", CompactionType.MAJOR);
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -566,8 +551,8 @@ public class TestWorker extends Compacto
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
-        Assert.assertEquals(1248L, buckets[0].getLen());
-        Assert.assertEquals(1248L, buckets[1].getLen());
+        Assert.assertEquals(624L, buckets[0].getLen());
+        Assert.assertEquals(624L, buckets[1].getLen());
       } else {
         LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName());
       }
@@ -580,18 +565,16 @@ public class TestWorker extends Compacto
     LOG.debug("Starting minorTableLegacy");
     Table t = newTable("default", "mtl", false);
 
-    HiveConf conf = new HiveConf();
-
-    addLegacyFile(conf, t, null, 20);
-    addDeltaFile(conf, t, null, 21L, 22L, 2);
-    addDeltaFile(conf, t, null, 23L, 24L, 2);
+    addLegacyFile(t, null, 20);
+    addDeltaFile(t, null, 21L, 22L, 2);
+    addDeltaFile(t, null, 23L, 24L, 2);
 
     burnThroughTransactions(25);
 
     CompactionRequest rqst = new CompactionRequest("default", "mtl", CompactionType.MINOR);
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -622,11 +605,11 @@ public class TestWorker extends Compacto
   public void majorPartitionWithBaseMissingBuckets() throws Exception {
     Table t = newTable("default", "mapwbmb", true);
     Partition p = newPartition(t, "today");
-    HiveConf conf = new HiveConf();
 
-    addBaseFile(conf, t, p, 20L, 20, 2, false);
-    addDeltaFile(conf, t, p, 21L, 22L, 2, 2, false);
-    addDeltaFile(conf, t, p, 23L, 24L, 2);
+
+    addBaseFile(t, p, 20L, 20, 2, false);
+    addDeltaFile(t, p, 21L, 22L, 2, 2, false);
+    addDeltaFile(t, p, 23L, 26L, 4);
 
     burnThroughTransactions(25);
 
@@ -634,7 +617,7 @@ public class TestWorker extends Compacto
     rqst.setPartitionname("ds=today");
     txnHandler.compact(rqst);
 
-    startWorker(new HiveConf());
+    startWorker();
 
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
     List<ShowCompactResponseElement> compacts = rsp.getCompacts();
@@ -649,7 +632,7 @@ public class TestWorker extends Compacto
     // Find the new delta file and make sure it has the right contents
     boolean sawNewBase = false;
     for (int i = 0; i < stat.length; i++) {
-      if (stat[i].getPath().getName().equals("base_0000024")) {
+      if (stat[i].getPath().getName().equals("base_0000026")) {
         sawNewBase = true;
         FileStatus[] buckets = fs.listStatus(stat[i].getPath());
         Assert.assertEquals(2, buckets.length);
@@ -658,10 +641,12 @@ public class TestWorker extends Compacto
         // Bucket 0 should be small and bucket 1 should be large, make sure that's the case
         Assert.assertTrue(
             ("bucket_00000".equals(buckets[0].getPath().getName()) && 104L == buckets[0].getLen()
-            && "bucket_00001".equals(buckets[1].getPath().getName()) && 1248L == buckets[1] .getLen())
+            && "bucket_00001".equals(buckets[1].getPath().getName()) && 676L == buckets[1]
+                .getLen())
             ||
             ("bucket_00000".equals(buckets[1].getPath().getName()) && 104L == buckets[1].getLen()
-            && "bucket_00001".equals(buckets[0].getPath().getName()) && 1248L == buckets[0] .getLen())
+            && "bucket_00001".equals(buckets[0].getPath().getName()) && 676L == buckets[0]
+                .getLen())
         );
       } else {
         LOG.debug("This is not the file you are looking for " + stat[i].getPath().getName());
@@ -669,9 +654,4 @@ public class TestWorker extends Compacto
     }
     Assert.assertTrue(sawNewBase);
   }
-
-  @Before
-  public void setUpTxnDb() throws Exception {
-    TxnDbUtil.setConfValues(new HiveConf());
-  }
 }

Modified: hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMath.java
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMath.java?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMath.java (original)
+++ hive/branches/llap/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFMath.java Tue Oct 14 19:06:45 2014
@@ -98,14 +98,6 @@ public class TestUDFMath {
     input = createDecimal("7.38905609893065");
     DoubleWritable res = udf.evaluate(input);
     Assert.assertEquals(2.0, res.get(), 0.000001);
-    
-    DoubleWritable input = new DoubleWritable(9.0);
-    res = udf.evaluate(createDecimal("3.0"), input);
-    Assert.assertEquals(2.0, res.get(), 0.000001);
-
-    DoubleWritable base = new DoubleWritable(3.0);
-    res = udf.evaluate(base, createDecimal("9.0"));
-    Assert.assertEquals(2.0, res.get(), 0.000001);
 
     res = udf.evaluate(createDecimal("3.0"), createDecimal("9.0"));
     Assert.assertEquals(2.0, res.get(), 0.000001);

Modified: hive/branches/llap/ql/src/test/queries/clientnegative/acid_overwrite.q
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/queries/clientnegative/acid_overwrite.q?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/queries/clientnegative/acid_overwrite.q (original)
+++ hive/branches/llap/ql/src/test/queries/clientnegative/acid_overwrite.q Tue Oct 14 19:06:45 2014
@@ -1,9 +1,8 @@
 set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.enforce.bucketing=true;
 
-create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc;
+create table acid_uanp(a int, b varchar(128)) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into table acid_uanp select cint, cast(cstring1 as varchar(128)) from alltypesorc where cint < 0 order by cint limit 10;
 insert overwrite table acid_uanp select cint, cast(cstring1 as varchar(128)) from alltypesorc;

Modified: hive/branches/llap/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q (original)
+++ hive/branches/llap/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q Tue Oct 14 19:06:45 2014
@@ -7,6 +7,6 @@ dfs ${system:test.dfs.mkdir} ${system:te
 dfs -touchz ${system:test.tmp.dir}/a_uri_crtab1/1.txt;
 dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab1/1.txt;
 
-create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext';
+create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab1';
 
 -- Attempt to create table with dir that does not have write permission should fail

Modified: hive/branches/llap/ql/src/test/queries/clientnegative/update_partition_col.q
URL: http://svn.apache.org/viewvc/hive/branches/llap/ql/src/test/queries/clientnegative/update_partition_col.q?rev=1631841&r1=1631840&r2=1631841&view=diff
==============================================================================
--- hive/branches/llap/ql/src/test/queries/clientnegative/update_partition_col.q (original)
+++ hive/branches/llap/ql/src/test/queries/clientnegative/update_partition_col.q Tue Oct 14 19:06:45 2014
@@ -1,8 +1,7 @@
 set hive.support.concurrency=true;
 set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.enforce.bucketing=true;
 
-create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc;
+create table foo(a int, b varchar(128)) partitioned by (ds string) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 update foo set ds = 'fred';