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

svn commit: r1640263 [4/12] - in /hive/branches/spark: ./ accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/ accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/mr/ accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/predicate/ accu...

Modified: hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java (original)
+++ hive/branches/spark/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java Tue Nov 18 00:48:40 2014
@@ -57,35 +57,27 @@ public class TestHiveDecimal {
     Assert.assertEquals("-1786135888657847525803324040144343378.1", dec.toString());
 
     dec = HiveDecimal.create("005.34000");
-    Assert.assertEquals(dec.precision(), 6);
-    Assert.assertEquals(dec.scale(), 5);
+    Assert.assertEquals(dec.precision(), 3);
+    Assert.assertEquals(dec.scale(), 2);
 
     dec = HiveDecimal.create("178613588865784752580332404014434337809799306448796128931113691624");
     Assert.assertNull(dec);
 
-    // Leaving trailing zeros
-    Assert.assertEquals("0.0", HiveDecimal.enforcePrecisionScale(new BigDecimal("0.0"), 2, 1).toString());
-    Assert.assertEquals("0.00", HiveDecimal.enforcePrecisionScale(new BigDecimal("0.00"), 3, 2).toString());
-    Assert.assertEquals("0.0000", HiveDecimal.enforcePrecisionScale(new BigDecimal("0.0000"), 10, 4).toString());
-    Assert.assertEquals("100.00000", HiveDecimal.enforcePrecisionScale(new BigDecimal("100.00000"), 15, 5).toString());
-    Assert.assertEquals("100.00", HiveDecimal.enforcePrecisionScale(new BigDecimal("100.00"), 15, 5).toString());
-
-    // Rounding numbers
-    Assert.assertEquals("0.01", HiveDecimal.enforcePrecisionScale(new BigDecimal("0.012"), 3, 2).toString());
-    Assert.assertEquals("0.02", HiveDecimal.enforcePrecisionScale(new BigDecimal("0.015"), 3, 2).toString());
-    Assert.assertEquals("0.01", HiveDecimal.enforcePrecisionScale(new BigDecimal("0.0145"), 3, 2).toString());
-
     // Rounding numbers that increase int digits
     Assert.assertEquals("10",
         HiveDecimal.enforcePrecisionScale(new BigDecimal("9.5"), 2, 0).toString());
     Assert.assertNull(HiveDecimal.enforcePrecisionScale(new BigDecimal("9.5"), 1, 0));
     Assert.assertEquals("9",
         HiveDecimal.enforcePrecisionScale(new BigDecimal("9.4"), 1, 0).toString());
+  }
 
-    // Integers with no scale values are not modified (zeros are not null)
-    Assert.assertEquals("0", HiveDecimal.enforcePrecisionScale(new BigDecimal("0"), 1, 0).toString());
-    Assert.assertEquals("30", HiveDecimal.enforcePrecisionScale(new BigDecimal("30"), 2, 0).toString());
-    Assert.assertEquals("5", HiveDecimal.enforcePrecisionScale(new BigDecimal("5"), 3, 2).toString());
+  @Test
+  @Concurrent(count=4)
+  @Repeating(repetition=100)
+  public void testTrailingZeroRemovalAfterEnforcement() {
+    String decStr = "8.090000000000000000000000000000000000000123456";
+    HiveDecimal dec = HiveDecimal.create(decStr);
+    Assert.assertEquals("8.09", dec.toString());
   }
 
   @Test
@@ -94,7 +86,7 @@ public class TestHiveDecimal {
   public void testMultiply() {
     HiveDecimal dec1 = HiveDecimal.create("0.00001786135888657847525803");
     HiveDecimal dec2 = HiveDecimal.create("3.0000123456789");
-    Assert.assertNotNull(dec1.multiply(dec2));
+    Assert.assertNull(dec1.multiply(dec2));
 
     dec1 = HiveDecimal.create("178613588865784752580323232232323444.4");
     dec2 = HiveDecimal.create("178613588865784752580302323232.3");
@@ -106,11 +98,11 @@ public class TestHiveDecimal {
 
     dec1 = HiveDecimal.create("3.140");
     dec2 = HiveDecimal.create("1.00");
-    Assert.assertEquals("3.14000", dec1.multiply(dec2).toString());
+    Assert.assertEquals("3.14", dec1.multiply(dec2).toString());
 
     dec1 = HiveDecimal.create("43.010");
     dec2 = HiveDecimal.create("2");
-    Assert.assertEquals("86.020", dec1.multiply(dec2).toString());
+    Assert.assertEquals("86.02", dec1.multiply(dec2).toString());
   }
 
   @Test
@@ -125,7 +117,7 @@ public class TestHiveDecimal {
     Assert.assertNull(dec1);
 
     dec1 = HiveDecimal.create("3.140");
-    Assert.assertEquals("9.859600", dec1.pow(2).toString());
+    Assert.assertEquals("9.8596", dec1.pow(2).toString());
   }
 
   @Test
@@ -155,7 +147,7 @@ public class TestHiveDecimal {
 
     dec1 = HiveDecimal.create("3.140");
     dec2 = HiveDecimal.create("1.00");
-    Assert.assertEquals("4.140", dec1.add(dec2).toString());
+    Assert.assertEquals("4.14", dec1.add(dec2).toString());
   }
 
 
@@ -165,7 +157,7 @@ public class TestHiveDecimal {
   public void testSubtract() {
       HiveDecimal dec1 = HiveDecimal.create("3.140");
       HiveDecimal dec2 = HiveDecimal.create("1.00");
-      Assert.assertEquals("2.140", dec1.subtract(dec2).toString());
+      Assert.assertEquals("2.14", dec1.subtract(dec2).toString());
   }
 
   @Test

Modified: hive/branches/spark/dev-support/jenkins-common.sh
URL: http://svn.apache.org/viewvc/hive/branches/spark/dev-support/jenkins-common.sh?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/dev-support/jenkins-common.sh (original)
+++ hive/branches/spark/dev-support/jenkins-common.sh Tue Nov 18 00:48:40 2014
@@ -22,10 +22,11 @@ fail() {
 # Exports two variables of import:
 # * BUILD_PROFILE - the profile which the ptest server understands
 # * BUILD_OPTS - additional test options to be sent to ptest cli
+# * PATCH_URL - the URL to the patch file
 process_jira() {
   test -n "$BRANCH" || fail "BRANCH must be specified"
   test -n "$JIRA_ROOT_URL" || fail "JIRA_ROOT_URL must be specified"
-  test -n "$JIRA_NAME" || fail "API_PASSWORD must be specified"
+  test -n "$JIRA_NAME" || fail "JIRA_NAME must be specified"
   JIRA_TEXT=$(mktemp)
   trap "rm -f $JIRA_TEXT" EXIT
   curl -s -S --location --retry 3 "${JIRA_ROOT_URL}/jira/browse/${JIRA_NAME}" > $JIRA_TEXT
@@ -39,7 +40,7 @@ process_jira() {
     fail "$JIRA_NAME is not \"Patch Available\". Exiting."
   fi
   # pull attachments from JIRA (hack stolen from hadoop since rest api doesn't show attachments)
-  PATCH_URL=$(grep -o '"/jira/secure/attachment/[0-9]*/[^"]*' $JIRA_TEXT | \
+  export PATCH_URL=$(grep -o '"/jira/secure/attachment/[0-9]*/[^"]*' $JIRA_TEXT | \
     grep -v -e 'htm[l]*$' | sort | tail -1 | \
     grep -o '/jira/secure/attachment/[0-9]*/[^"]*')
   if [[ -z "$PATCH_URL" ]]

Modified: hive/branches/spark/dev-support/jenkins-execute-build.sh
URL: http://svn.apache.org/viewvc/hive/branches/spark/dev-support/jenkins-execute-build.sh?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/dev-support/jenkins-execute-build.sh (original)
+++ hive/branches/spark/dev-support/jenkins-execute-build.sh Tue Nov 18 00:48:40 2014
@@ -19,15 +19,34 @@ test -n "$BRANCH" || fail "BRANCH must b
 test -n "$API_ENDPOINT" || fail "API_ENDPOINT must be specified"
 test -n "$LOG_ENDPOINT" || fail "LOG_ENDPOINT must be specified"
 test -n "$API_PASSWORD" || fail "API_PASSWORD must be specified"
-export JIRA_NAME="HIVE-${ISSUE_NUM}"
+if [[ -n "$ISSUE_NUM" ]]
+then
+  export JIRA_NAME="HIVE-${ISSUE_NUM}"
+fi
 export ROOT=$PWD
 export JIRA_ROOT_URL="https://issues.apache.org"
 export BUILD_TAG="${BUILD_TAG##jenkins-}"
-echo $JIRA_NAME
+if [[ -n "$JIRA_NAME" ]]
+then
+  echo $JIRA_NAME
+fi
 set -x
 env
 
-process_jira
+if [[ -n "$JIRA_NAME" ]]
+then
+  process_jira
+fi
+
+profile=$BUILD_PROFILE
+if [[ -z "$profile" ]]
+then
+  profile=$DEFAULT_BUILD_PROFILE
+fi
+if [[ -z "$profile" ]]
+then
+  fail "Could not find build profile"
+fi
 
 test -d hive/build/ || mkdir -p hive/build/
 cd hive/build/
@@ -35,17 +54,13 @@ rm -rf ptest2
 svn co http://svn.apache.org/repos/asf/hive/trunk/testutils/ptest2/ ptest2
 cd ptest2
 
-# sanity check the profile
-case "$BUILD_PROFILE" in
-  trunk-mr1);;
-  trunk-mr2);;
-  *)
-  echo "Unknown profile '$BUILD_PROFILE'"
-  exit 1
-  ;;
-esac
 mvn clean package -DskipTests -Drat.numUnapprovedLicenses=1000 -Dmaven.repo.local=$WORKSPACE/.m2
 set +e
+optionalArgs=()
+if [[ -n "$JIRA_NAME" ]]
+then
+  optionalArgs=(--patch "${JIRA_ROOT_URL}${PATCH_URL}" --jira "$JIRA_NAME")
+fi
 java -cp "target/hive-ptest-1.0-classes.jar:target/lib/*" org.apache.hive.ptest.api.client.PTestClient --endpoint "$API_ENDPOINT" \
   --logsEndpoint "$LOG_ENDPOINT" \
   --command testStart \
@@ -53,8 +68,7 @@ java -cp "target/hive-ptest-1.0-classes.
   --password $API_PASSWORD \
   --outputDir target/ \
   --testHandle "$BUILD_TAG" \
-  --patch "${JIRA_ROOT_URL}${PATCH_URL}" \
-  --jira "$JIRA_NAME" ${BUILD_OPTS} "$@"
+  ${optionalArgs[@]} ${BUILD_OPTS} "$@"
 ret=$?
 cd target/
 if [[ -f test-results.tar.gz ]]

Modified: hive/branches/spark/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseKeyFactory3.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseKeyFactory3.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseKeyFactory3.java (original)
+++ hive/branches/spark/hbase-handler/src/test/org/apache/hadoop/hive/hbase/TestHBaseKeyFactory3.java Tue Nov 18 00:48:40 2014
@@ -1,3 +1,21 @@
+/**
+ * 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.hadoop.hive.hbase;
 
 import java.io.IOException;

Modified: hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java (original)
+++ hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HCatUtil.java Tue Nov 18 00:48:40 2014
@@ -84,8 +84,9 @@ public class HCatUtil {
   private static volatile HiveClientCache hiveClientCache;
 
   public static boolean checkJobContextIfRunningFromBackend(JobContext j) {
-    if (j.getConfiguration().get("mapred.task.id", "").equals("") &&
-        !("true".equals(j.getConfiguration().get("pig.illustrating")))) {
+    if (j.getConfiguration().get("pig.job.converted.fetch", "").equals("") &&
+          j.getConfiguration().get("mapred.task.id", "").equals("") &&
+          !("true".equals(j.getConfiguration().get("pig.illustrating")))) {
       return false;
     }
     return true;

Modified: hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java (original)
+++ hive/branches/spark/hcatalog/core/src/main/java/org/apache/hive/hcatalog/common/HiveClientCache.java Tue Nov 18 00:48:40 2014
@@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentMa
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -44,6 +45,7 @@ import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.RemovalListener;
 import com.google.common.cache.RemovalNotification;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * A thread safe time expired cache for HiveMetaStoreClient
@@ -59,7 +61,7 @@ class HiveClientCache {
 
   private static final AtomicInteger nextId = new AtomicInteger(0);
 
-  private ScheduledFuture<?> cleanupHandle; // used to cleanup cache
+  private final ScheduledFuture<?> cleanupHandle; // used to cleanup cache
 
   // Since HiveMetaStoreClient is not threadsafe, hive clients are not  shared across threads.
   // Thread local variable containing each thread's unique ID, is used as one of the keys for the cache
@@ -91,6 +93,7 @@ class HiveClientCache {
     this.timeout = timeout;
     RemovalListener<HiveClientCacheKey, CacheableHiveMetaStoreClient> removalListener =
       new RemovalListener<HiveClientCacheKey, CacheableHiveMetaStoreClient>() {
+        @Override
         public void onRemoval(RemovalNotification<HiveClientCacheKey, CacheableHiveMetaStoreClient> notification) {
           CacheableHiveMetaStoreClient hiveMetaStoreClient = notification.getValue();
           if (hiveMetaStoreClient != null) {
@@ -108,6 +111,7 @@ class HiveClientCache {
 
     // Add a maintenance thread that will attempt to trigger a cache clean continuously
     Runnable cleanupThread = new Runnable() {
+      @Override
       public void run() {
         hiveCache.cleanUp();
       }
@@ -134,7 +138,10 @@ class HiveClientCache {
      * 5 seconds after the first timeout, and then after that, it'll check for whether or not
      * it can be cleaned every max(DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS,timeout) seconds
      */
-    cleanupHandle = Executors.newScheduledThreadPool(1).scheduleWithFixedDelay(
+    ThreadFactory daemonThreadFactory = (new ThreadFactoryBuilder()).setDaemon(true)
+        .setNameFormat("HiveClientCache-cleaner-%d").build();
+
+    cleanupHandle = Executors.newScheduledThreadPool(1, daemonThreadFactory).scheduleWithFixedDelay(
         cleanupThread,
         timeout + 5, cleanupInterval, TimeUnit.SECONDS);
 
@@ -285,7 +292,7 @@ class HiveClientCache {
    * Add # of current users on HiveMetaStoreClient, so that the client can be cleaned when no one is using it.
    */
   public static class CacheableHiveMetaStoreClient extends HiveMetaStoreClient {
-    private AtomicInteger users = new AtomicInteger(0);
+    private final AtomicInteger users = new AtomicInteger(0);
     private volatile boolean expiredFromCache = false;
     private boolean isClosed = false;
     private final long expiryTime;

Modified: hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataJSONSerializer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataJSONSerializer.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataJSONSerializer.java (original)
+++ hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataJSONSerializer.java Tue Nov 18 00:48:40 2014
@@ -1,3 +1,22 @@
+/**
+ * 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.api;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;

Modified: hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataSerializer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataSerializer.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataSerializer.java (original)
+++ hive/branches/spark/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/MetadataSerializer.java Tue Nov 18 00:48:40 2014
@@ -1,3 +1,22 @@
+/**
+ * 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.api;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;

Modified: hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java (original)
+++ hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java Tue Nov 18 00:48:40 2014
@@ -43,6 +43,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  */
@@ -229,9 +230,9 @@ public class TestCompactor {
     Worker t = new Worker();
     t.setThreadId((int) t.getId());
     t.setHiveConf(conf);
-    MetaStoreThread.BooleanPointer stop = new MetaStoreThread.BooleanPointer();
-    MetaStoreThread.BooleanPointer looped = new MetaStoreThread.BooleanPointer();
-    stop.boolVal = true;
+    AtomicBoolean stop = new AtomicBoolean();
+    AtomicBoolean looped = new AtomicBoolean();
+    stop.set(true);
     t.init(stop, looped);
     t.run();
     ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());

Modified: hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java (original)
+++ hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java Tue Nov 18 00:48:40 2014
@@ -529,7 +529,7 @@ public class TestBeeLineWithArgs {
   public void testQueryProgress() throws Throwable {
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
         "select count(*) from " + tableName + ";\n";
-    final String EXPECTED_PATTERN = "Parsing command";
+    final String EXPECTED_PATTERN = "number of splits";
     testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, getBaseArgs(miniHS2.getBaseJdbcURL()));
   }
 

Modified: hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java Tue Nov 18 00:48:40 2014
@@ -51,6 +51,7 @@ 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.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.processors.DfsProcessor;
@@ -105,6 +106,7 @@ public class TestJdbcDriver2 {
   public static void setUpBeforeClass() throws SQLException, ClassNotFoundException{
     Class.forName(driverName);
     Connection con1 = getConnection("default");
+    System.setProperty(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_VERBOSE.varname, "" + true);
 
     Statement stmt1 = con1.createStatement();
     assertNotNull("Statement is null", stmt1);

Modified: hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java (original)
+++ hive/branches/spark/itests/hive-unit/src/test/java/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java Tue Nov 18 00:48:40 2014
@@ -18,6 +18,7 @@
 
 package org.apache.hive.service.cli;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.cli.thrift.EmbeddedThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
@@ -35,6 +36,7 @@ public class TestEmbeddedThriftBinaryCLI
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     service = new EmbeddedThriftBinaryCLIService();
+    service.init(new HiveConf());
     client = new ThriftCLIServiceClient(service);
   }
 

Modified: hive/branches/spark/itests/qtest/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/qtest/pom.xml?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/qtest/pom.xml (original)
+++ hive/branches/spark/itests/qtest/pom.xml Tue Nov 18 00:48:40 2014
@@ -414,22 +414,6 @@
                 <mkdir dir="${project.build.directory}/qfile-results/contribclientpositive"/>
                 <mkdir dir="${project.build.directory}/qfile-results/contribclientnegative"/>
 
-
-                <!-- Parse -->
-                <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
-                  outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/ql/parse/"
-                  templatePath="${basedir}/${hive.path.to.root}/ql/src/test/templates/" template="TestParse.vm"
-                  queryDirectory="${basedir}/${hive.path.to.root}/ql/src/test/queries/positive/"
-                  queryFile="${qfile}"
-                  queryFileRegex="${qfile_regex}"
-                  runDisabled="${run_disabled}"
-                  resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/compiler/" className="TestParse"
-                  logFile="${project.build.directory}/testparsegen.log"
-                  hadoopVersion="${active.hadoop.version}"
-                  logDirectory="${project.build.directory}/qfile-results/positive/"
-                  initScript="q_test_init.sql"
-                  cleanupScript="q_test_cleanup.sql"/>
-
                 <!-- Negative Parse -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
                   outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/ql/parse/"

Modified: hive/branches/spark/itests/src/test/resources/testconfiguration.properties
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/src/test/resources/testconfiguration.properties?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/src/test/resources/testconfiguration.properties (original)
+++ hive/branches/spark/itests/src/test/resources/testconfiguration.properties Tue Nov 18 00:48:40 2014
@@ -108,6 +108,7 @@ minitez.query.files.shared=alter_merge_2
   insert_update_delete.q,\
   join0.q,\
   join1.q,\
+  join_nullsafe.q,\
   leftsemijoin.q,\
   limit_pushdown.q,\
   load_dyn_part1.q,\

Modified: hive/branches/spark/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/branches/spark/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java Tue Nov 18 00:48:40 2014
@@ -537,10 +537,11 @@ public class QTestUtil {
   /**
    * Clear out any side effects of running tests
    */
-  public void clearTestSideEffects() throws Exception {
+  public void clearTablesCreatedDuringTests() throws Exception {
     if (System.getenv(QTEST_LEAVE_FILES) != null) {
       return;
     }
+
     // Delete any tables other than the source tables
     // and any databases other than the default database.
     for (String dbName : db.getAllDatabases()) {
@@ -574,9 +575,11 @@ public class QTestUtil {
     try {
       Path p = new Path(testWarehouse);
       FileSystem fileSystem = p.getFileSystem(conf);
-      for (FileStatus status : fileSystem.listStatus(p)) {
-        if (status.isDir() && !srcTables.contains(status.getPath().getName())) {
-          fileSystem.delete(status.getPath(), true);
+      if (fileSystem.exists(p)) {
+        for (FileStatus status : fileSystem.listStatus(p)) {
+          if (status.isDir() && !srcTables.contains(status.getPath().getName())) {
+            fileSystem.delete(status.getPath(), true);
+          }
         }
       }
     } catch (IllegalArgumentException e) {
@@ -590,6 +593,18 @@ public class QTestUtil {
           db.dropRole(roleName);
         }
     }
+  }
+
+  /**
+   * Clear out any side effects of running tests
+   */
+  public void clearTestSideEffects() throws Exception {
+    if (System.getenv(QTEST_LEAVE_FILES) != null) {
+      return;
+    }
+
+    clearTablesCreatedDuringTests();
+
     // allocate and initialize a new conf since a test can
     // modify conf by using 'set' commands
     conf = new HiveConf (Driver.class);
@@ -606,6 +621,8 @@ public class QTestUtil {
       return;
     }
 
+    clearTablesCreatedDuringTests();
+
     SessionState.get().getConf().setBoolean("hive.test.shutdown.phase", true);
 
     String cleanupCommands = readEntireFileIntoString(new File(cleanupScript));

Modified: hive/branches/spark/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java (original)
+++ hive/branches/spark/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java Tue Nov 18 00:48:40 2014
@@ -52,6 +52,7 @@ import javax.security.sasl.SaslException
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
 import org.apache.hive.service.auth.HiveAuthFactory;
@@ -143,7 +144,9 @@ public class HiveConnection implements j
     isEmbeddedMode = connParams.isEmbeddedMode();
 
     if (isEmbeddedMode) {
-      client = new EmbeddedThriftBinaryCLIService();
+      EmbeddedThriftBinaryCLIService embeddedClient = new EmbeddedThriftBinaryCLIService();
+      embeddedClient.init(new HiveConf());
+      client = embeddedClient;
     } else {
       // extract user/password from JDBC connection properties if its not supplied in the
       // connection URL

Modified: hive/branches/spark/metastore/if/hive_metastore.thrift
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/if/hive_metastore.thrift?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/if/hive_metastore.thrift (original)
+++ hive/branches/spark/metastore/if/hive_metastore.thrift Tue Nov 18 00:48:40 2014
@@ -1139,6 +1139,7 @@ const string META_TABLE_PARTITION_COLUMN
 const string FILE_INPUT_FORMAT    = "file.inputformat",
 const string FILE_OUTPUT_FORMAT   = "file.outputformat",
 const string META_TABLE_STORAGE   = "storage_handler",
-
+const string TABLE_IS_TRANSACTIONAL = "transactional",
+const string TABLE_NO_AUTO_COMPACT = "no_auto_compaction",
 
 

Modified: hive/branches/spark/metastore/scripts/upgrade/oracle/020-HIVE-7784.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/scripts/upgrade/oracle/020-HIVE-7784.oracle.sql?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/scripts/upgrade/oracle/020-HIVE-7784.oracle.sql (original)
+++ hive/branches/spark/metastore/scripts/upgrade/oracle/020-HIVE-7784.oracle.sql Tue Nov 18 00:48:40 2014
@@ -1,27 +1,2 @@
---
--- Create the table if it doesn't exist.
---
-CREATE TABLE IF NOT EXISTS PART_COL_STATS (
- CS_ID NUMBER NOT NULL,
- DB_NAME VARCHAR2(128) NOT NULL,
- TABLE_NAME VARCHAR2(128) NOT NULL,
- PARTITION_NAME VARCHAR2(767) NOT NULL,
- COLUMN_NAME VARCHAR2(128) NOT NULL,
- COLUMN_TYPE VARCHAR2(128) NOT NULL,
- PART_ID NUMBER NOT NULL,
- LONG_LOW_VALUE NUMBER,
- LONG_HIGH_VALUE NUMBER,
- DOUBLE_LOW_VALUE NUMBER,
- DOUBLE_HIGH_VALUE NUMBER,
- BIG_DECIMAL_LOW_VALUE VARCHAR2(4000),
- BIG_DECIMAL_HIGH_VALUE VARCHAR2(4000),
- NUM_NULLS NUMBER NOT NULL,
- NUM_DISTINCTS NUMBER,
- AVG_COL_LEN NUMBER,
- MAX_COL_LEN NUMBER,
- NUM_TRUES NUMBER,
- NUM_FALSES NUMBER,
- LAST_ANALYZED NUMBER NOT NULL
-);
-
 CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME);
+

Modified: hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp (original)
+++ hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.cpp Tue Nov 18 00:48:40 2014
@@ -53,6 +53,10 @@ hive_metastoreConstants::hive_metastoreC
 
   META_TABLE_STORAGE = "storage_handler";
 
+  TABLE_IS_TRANSACTIONAL = "transactional";
+
+  TABLE_NO_AUTO_COMPACT = "no_auto_compaction";
+
 }
 
 }}} // namespace

Modified: hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h (original)
+++ hive/branches/spark/metastore/src/gen/thrift/gen-cpp/hive_metastore_constants.h Tue Nov 18 00:48:40 2014
@@ -36,6 +36,8 @@ class hive_metastoreConstants {
   std::string FILE_INPUT_FORMAT;
   std::string FILE_OUTPUT_FORMAT;
   std::string META_TABLE_STORAGE;
+  std::string TABLE_IS_TRANSACTIONAL;
+  std::string TABLE_NO_AUTO_COMPACT;
 };
 
 extern const hive_metastoreConstants g_hive_metastore_constants;

Modified: hive/branches/spark/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java (original)
+++ hive/branches/spark/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/hive_metastoreConstants.java Tue Nov 18 00:48:40 2014
@@ -75,4 +75,8 @@ public class hive_metastoreConstants {
 
   public static final String META_TABLE_STORAGE = "storage_handler";
 
+  public static final String TABLE_IS_TRANSACTIONAL = "transactional";
+
+  public static final String TABLE_NO_AUTO_COMPACT = "no_auto_compaction";
+
 }

Modified: hive/branches/spark/metastore/src/gen/thrift/gen-php/metastore/Types.php
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/gen/thrift/gen-php/metastore/Types.php?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/gen/thrift/gen-php/metastore/Types.php (original)
+++ hive/branches/spark/metastore/src/gen/thrift/gen-php/metastore/Types.php Tue Nov 18 00:48:40 2014
@@ -12890,4 +12890,8 @@ $GLOBALS['hive_metastore_CONSTANTS']['FI
 
 $GLOBALS['hive_metastore_CONSTANTS']['META_TABLE_STORAGE'] = "storage_handler";
 
+$GLOBALS['hive_metastore_CONSTANTS']['TABLE_IS_TRANSACTIONAL'] = "transactional";
+
+$GLOBALS['hive_metastore_CONSTANTS']['TABLE_NO_AUTO_COMPACT'] = "no_auto_compaction";
+
 

Modified: hive/branches/spark/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py (original)
+++ hive/branches/spark/metastore/src/gen/thrift/gen-py/hive_metastore/constants.py Tue Nov 18 00:48:40 2014
@@ -30,3 +30,5 @@ META_TABLE_PARTITION_COLUMN_TYPES = "par
 FILE_INPUT_FORMAT = "file.inputformat"
 FILE_OUTPUT_FORMAT = "file.outputformat"
 META_TABLE_STORAGE = "storage_handler"
+TABLE_IS_TRANSACTIONAL = "transactional"
+TABLE_NO_AUTO_COMPACT = "no_auto_compaction"

Modified: hive/branches/spark/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb (original)
+++ hive/branches/spark/metastore/src/gen/thrift/gen-rb/hive_metastore_constants.rb Tue Nov 18 00:48:40 2014
@@ -49,3 +49,7 @@ FILE_OUTPUT_FORMAT = %q"file.outputforma
 
 META_TABLE_STORAGE = %q"storage_handler"
 
+TABLE_IS_TRANSACTIONAL = %q"transactional"
+
+TABLE_NO_AUTO_COMPACT = %q"no_auto_compaction"
+

Modified: hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Tue Nov 18 00:48:40 2014
@@ -43,6 +43,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.Timer;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -5720,7 +5721,7 @@ public class HiveMetaStore extends Thrif
 
       Lock startLock = new ReentrantLock();
       Condition startCondition = startLock.newCondition();
-      MetaStoreThread.BooleanPointer startedServing = new MetaStoreThread.BooleanPointer();
+      AtomicBoolean startedServing = new AtomicBoolean();
       startMetaStoreThreads(conf, startLock, startCondition, startedServing);
       startMetaStore(cli.port, ShimLoader.getHadoopThriftAuthBridge(), conf, startLock,
           startCondition, startedServing);
@@ -5767,7 +5768,7 @@ public class HiveMetaStore extends Thrif
    */
   public static void startMetaStore(int port, HadoopThriftAuthBridge bridge,
       HiveConf conf, Lock startLock, Condition startCondition,
-      MetaStoreThread.BooleanPointer startedServing) throws Throwable {
+      AtomicBoolean startedServing) throws Throwable {
     try {
       isMetaStoreRemote = true;
       // Server will create new threads up to max as necessary. After an idle
@@ -5851,7 +5852,7 @@ public class HiveMetaStore extends Thrif
 
   private static void signalOtherThreadsToStart(final TServer server, final Lock startLock,
                                                 final Condition startCondition,
-                                                final MetaStoreThread.BooleanPointer startedServing) {
+                                                final AtomicBoolean startedServing) {
     // A simple thread to wait until the server has started and then signal the other threads to
     // begin
     Thread t = new Thread() {
@@ -5866,7 +5867,7 @@ public class HiveMetaStore extends Thrif
         } while (!server.isServing());
         startLock.lock();
         try {
-          startedServing.boolVal = true;
+          startedServing.set(true);
           startCondition.signalAll();
         } finally {
           startLock.unlock();
@@ -5882,7 +5883,7 @@ public class HiveMetaStore extends Thrif
    */
   private static void startMetaStoreThreads(final HiveConf conf, final Lock startLock,
                                             final Condition startCondition, final
-                                            MetaStoreThread.BooleanPointer startedServing) {
+                                            AtomicBoolean startedServing) {
     // A thread is spun up to start these other threads.  That's because we can't start them
     // until after the TServer has started, but once TServer.serve is called we aren't given back
     // control.
@@ -5900,7 +5901,7 @@ public class HiveMetaStore extends Thrif
         try {
           // Per the javadocs on Condition, do not depend on the condition alone as a start gate
           // since spurious wake ups are possible.
-          while (!startedServing.boolVal) startCondition.await();
+          while (!startedServing.get()) startCondition.await();
           startCompactorInitiator(conf);
           startCompactorWorkers(conf);
           startCompactorCleaner(conf);
@@ -5960,7 +5961,7 @@ public class HiveMetaStore extends Thrif
     LOG.info("Starting metastore thread of type " + thread.getClass().getName());
     thread.setHiveConf(conf);
     thread.setThreadId(nextThreadId++);
-    thread.init(new MetaStoreThread.BooleanPointer(), new MetaStoreThread.BooleanPointer());
+    thread.init(new AtomicBoolean(), new AtomicBoolean());
     thread.start();
   }
 }

Modified: hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java?rev=1640263&r1=1640262&r2=1640263&view=diff
==============================================================================
--- hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java (original)
+++ hive/branches/spark/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java Tue Nov 18 00:48:40 2014
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hive.met
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.isIndexTable;
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -147,6 +148,7 @@ public class HiveMetaStoreClient impleme
   protected final HiveConf conf;
   private String tokenStrForm;
   private final boolean localMetaStore;
+  private final MetaStoreFilterHook filterHook;
 
   private Map<String, String> currentMetaVars;
 
@@ -169,6 +171,7 @@ public class HiveMetaStoreClient impleme
       conf = new HiveConf(HiveMetaStoreClient.class);
     }
     this.conf = conf;
+    filterHook = loadFilterHooks();
 
     String msUri = conf.getVar(HiveConf.ConfVars.METASTOREURIS);
     localMetaStore = HiveConfUtil.isEmbeddedMetaStore(msUri);
@@ -215,6 +218,31 @@ public class HiveMetaStoreClient impleme
     open();
   }
 
+  private MetaStoreFilterHook loadFilterHooks() throws IllegalStateException {
+    Class<? extends MetaStoreFilterHook> authProviderClass = conf.
+        getClass(HiveConf.ConfVars.METASTORE_FILTER_HOOK.varname,
+            DefaultMetaStoreFilterHookImpl.class,
+            MetaStoreFilterHook.class);
+    String msg = "Unable to create instance of " + authProviderClass.getName() + ": ";
+    try {
+      Constructor<? extends MetaStoreFilterHook> constructor =
+          authProviderClass.getConstructor(HiveConf.class);
+      return constructor.newInstance(conf);
+    } catch (NoSuchMethodException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    } catch (SecurityException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    } catch (InstantiationException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    } catch (IllegalArgumentException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    } catch (InvocationTargetException e) {
+      throw new IllegalStateException(msg + e.getMessage(), e);
+    }
+  }
+
   /**
    * Swaps the first element of the metastoreUris array with a random element from the
    * remainder of the array.
@@ -498,7 +526,7 @@ public class HiveMetaStoreClient impleme
         part.getDbName(), part.getTableName(), parts, ifNotExists);
     req.setNeedResult(needResults);
     AddPartitionsResult result = client.add_partitions_req(req);
-    return needResults ? result.getPartitions() : null;
+    return needResults ? filterHook.filterPartitions(result.getPartitions()) : null;
   }
 
   @Override
@@ -667,7 +695,7 @@ public class HiveMetaStoreClient impleme
        for (String table : tableList) {
          try {
            // Subclasses can override this step (for example, for temporary tables)
-           dropTable(name, table, deleteData, false);
+           dropTable(name, table, deleteData, true);
          } catch (UnsupportedOperationException e) {
            // Ignore Index tables, those will be dropped with parent tables
          }
@@ -904,7 +932,7 @@ public class HiveMetaStoreClient impleme
   public List<String> getDatabases(String databasePattern)
     throws MetaException {
     try {
-      return client.get_databases(databasePattern);
+      return filterHook.filterDatabases(client.get_databases(databasePattern));
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -915,7 +943,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<String> getAllDatabases() throws MetaException {
     try {
-      return client.get_all_databases();
+      return filterHook.filterDatabases(client.get_all_databases());
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -934,29 +962,30 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<Partition> listPartitions(String db_name, String tbl_name,
       short max_parts) throws NoSuchObjectException, MetaException, TException {
-    return deepCopyPartitions(
-        client.get_partitions(db_name, tbl_name, max_parts));
+    return deepCopyPartitions(filterHook.filterPartitions(
+        client.get_partitions(db_name, tbl_name, max_parts)));
   }
 
   @Override
   public PartitionSpecProxy listPartitionSpecs(String dbName, String tableName, int maxParts) throws TException {
-    return PartitionSpecProxy.Factory.get(client.get_partitions_pspec(dbName, tableName, maxParts));
+    return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs(
+        client.get_partitions_pspec(dbName, tableName, maxParts)));
   }
 
   @Override
   public List<Partition> listPartitions(String db_name, String tbl_name,
       List<String> part_vals, short max_parts)
       throws NoSuchObjectException, MetaException, TException {
-    return deepCopyPartitions(
-        client.get_partitions_ps(db_name, tbl_name, part_vals, max_parts));
+    return deepCopyPartitions(filterHook.filterPartitions(
+        client.get_partitions_ps(db_name, tbl_name, part_vals, max_parts)));
   }
 
   @Override
   public List<Partition> listPartitionsWithAuthInfo(String db_name,
       String tbl_name, short max_parts, String user_name, List<String> group_names)
        throws NoSuchObjectException, MetaException, TException {
-    return deepCopyPartitions(
-        client.get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names));
+    return deepCopyPartitions(filterHook.filterPartitions(
+        client.get_partitions_with_auth(db_name, tbl_name, max_parts, user_name, group_names)));
   }
 
   @Override
@@ -964,8 +993,8 @@ public class HiveMetaStoreClient impleme
       String tbl_name, List<String> part_vals, short max_parts,
       String user_name, List<String> group_names) throws NoSuchObjectException,
       MetaException, TException {
-    return deepCopyPartitions(client.get_partitions_ps_with_auth(db_name,
-        tbl_name, part_vals, max_parts, user_name, group_names));
+    return deepCopyPartitions(filterHook.filterPartitions(client.get_partitions_ps_with_auth(db_name,
+        tbl_name, part_vals, max_parts, user_name, group_names)));
   }
 
   /**
@@ -986,16 +1015,16 @@ public class HiveMetaStoreClient impleme
   public List<Partition> listPartitionsByFilter(String db_name, String tbl_name,
       String filter, short max_parts) throws MetaException,
          NoSuchObjectException, TException {
-    return deepCopyPartitions(
-        client.get_partitions_by_filter(db_name, tbl_name, filter, max_parts));
+    return deepCopyPartitions(filterHook.filterPartitions(
+        client.get_partitions_by_filter(db_name, tbl_name, filter, max_parts)));
   }
 
   @Override
   public PartitionSpecProxy listPartitionSpecsByFilter(String db_name, String tbl_name,
                                                        String filter, int max_parts) throws MetaException,
          NoSuchObjectException, TException {
-    return PartitionSpecProxy.Factory.get(
-        client.get_part_specs_by_filter(db_name, tbl_name, filter, max_parts));
+    return PartitionSpecProxy.Factory.get(filterHook.filterPartitionSpecs(
+        client.get_part_specs_by_filter(db_name, tbl_name, filter, max_parts)));
   }
 
   @Override
@@ -1023,6 +1052,7 @@ public class HiveMetaStoreClient impleme
       throw new IncompatibleMetastoreException(
           "Metastore doesn't support listPartitionsByExpr: " + te.getMessage());
     }
+    r.setPartitions(filterHook.filterPartitions(r.getPartitions()));
     // TODO: in these methods, do we really need to deepcopy?
     deepCopyPartitions(r.getPartitions(), result);
     return !r.isSetHasUnknownPartitions() || r.isHasUnknownPartitions(); // Assume the worst.
@@ -1040,7 +1070,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public Database getDatabase(String name) throws NoSuchObjectException,
       MetaException, TException {
-    return deepCopy(client.get_database(name));
+    return deepCopy(filterHook.filterDatabase(client.get_database(name)));
   }
 
   /**
@@ -1056,13 +1086,15 @@ public class HiveMetaStoreClient impleme
   @Override
   public Partition getPartition(String db_name, String tbl_name,
       List<String> part_vals) throws NoSuchObjectException, MetaException, TException {
-    return deepCopy(client.get_partition(db_name, tbl_name, part_vals));
+    return deepCopy(filterHook.filterPartition(
+        client.get_partition(db_name, tbl_name, part_vals)));
   }
 
   @Override
   public List<Partition> getPartitionsByNames(String db_name, String tbl_name,
       List<String> part_names) throws NoSuchObjectException, MetaException, TException {
-    return deepCopyPartitions(client.get_partitions_by_names(db_name, tbl_name, part_names));
+    return deepCopyPartitions(filterHook.filterPartitions(
+        client.get_partitions_by_names(db_name, tbl_name, part_names)));
   }
 
   @Override
@@ -1070,8 +1102,8 @@ public class HiveMetaStoreClient impleme
       List<String> part_vals, String user_name, List<String> group_names)
       throws MetaException, UnknownTableException, NoSuchObjectException,
       TException {
-    return deepCopy(client.get_partition_with_auth(db_name, tbl_name, part_vals, user_name,
-        group_names));
+    return deepCopy(filterHook.filterPartition(client.get_partition_with_auth(db_name,
+        tbl_name, part_vals, user_name, group_names)));
   }
 
   /**
@@ -1088,7 +1120,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public Table getTable(String dbname, String name) throws MetaException,
       TException, NoSuchObjectException {
-    return deepCopy(client.get_table(dbname, name));
+    return deepCopy(filterHook.filterTable(client.get_table(dbname, name)));
   }
 
   /** {@inheritDoc} */
@@ -1096,21 +1128,23 @@ public class HiveMetaStoreClient impleme
   @Deprecated
   public Table getTable(String tableName) throws MetaException, TException,
       NoSuchObjectException {
-    return getTable(DEFAULT_DATABASE_NAME, tableName);
+    return filterHook.filterTable(getTable(DEFAULT_DATABASE_NAME, tableName));
   }
 
   /** {@inheritDoc} */
   @Override
   public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
       throws MetaException, InvalidOperationException, UnknownDBException, TException {
-    return deepCopyTables(client.get_table_objects_by_name(dbName, tableNames));
+    return deepCopyTables(filterHook.filterTables(
+        client.get_table_objects_by_name(dbName, tableNames)));
   }
 
   /** {@inheritDoc} */
   @Override
   public List<String> listTableNamesByFilter(String dbName, String filter, short maxTables)
       throws MetaException, TException, InvalidOperationException, UnknownDBException {
-    return client.get_table_names_by_filter(dbName, filter, maxTables);
+    return filterHook.filterTableNames(dbName,
+        client.get_table_names_by_filter(dbName, filter, maxTables));
   }
 
   /**
@@ -1129,7 +1163,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<String> getTables(String dbname, String tablePattern) throws MetaException {
     try {
-      return client.get_tables(dbname, tablePattern);
+      return filterHook.filterTableNames(dbname, client.get_tables(dbname, tablePattern));
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -1140,7 +1174,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<String> getAllTables(String dbname) throws MetaException {
     try {
-      return client.get_all_tables(dbname);
+      return filterHook.filterTableNames(dbname, client.get_all_tables(dbname));
     } catch (Exception e) {
       MetaStoreUtils.logAndThrowMetaException(e);
     }
@@ -1151,11 +1185,10 @@ public class HiveMetaStoreClient impleme
   public boolean tableExists(String databaseName, String tableName) throws MetaException,
       TException, UnknownDBException {
     try {
-      client.get_table(databaseName, tableName);
+      return filterHook.filterTable(client.get_table(databaseName, tableName)) == null;
     } catch (NoSuchObjectException e) {
       return false;
     }
-    return true;
   }
 
   /** {@inheritDoc} */
@@ -1169,14 +1202,16 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<String> listPartitionNames(String dbName, String tblName,
       short max) throws MetaException, TException {
-    return client.get_partition_names(dbName, tblName, max);
+    return filterHook.filterPartitionNames(dbName, tblName, 
+        client.get_partition_names(dbName, tblName, max));
   }
 
   @Override
   public List<String> listPartitionNames(String db_name, String tbl_name,
       List<String> part_vals, short max_parts)
       throws MetaException, TException, NoSuchObjectException {
-    return client.get_partition_names_ps(db_name, tbl_name, part_vals, max_parts);
+    return filterHook.filterPartitionNames(db_name, tbl_name, 
+        client.get_partition_names_ps(db_name, tbl_name, part_vals, max_parts));
   }
 
   @Override
@@ -1259,7 +1294,7 @@ public class HiveMetaStoreClient impleme
   public Index getIndex(String dbName, String tblName, String indexName)
       throws MetaException, UnknownTableException, NoSuchObjectException,
       TException {
-    return deepCopy(client.get_index_by_name(dbName, tblName, indexName));
+    return deepCopy(filterHook.filterIndex(client.get_index_by_name(dbName, tblName, indexName)));
   }
 
   /**
@@ -1275,7 +1310,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<String> listIndexNames(String dbName, String tblName, short max)
       throws MetaException, TException {
-    return client.get_index_names(dbName, tblName, max);
+    return filterHook.filterIndexNames(dbName, tblName, client.get_index_names(dbName, tblName, max));
   }
 
   /**
@@ -1291,7 +1326,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public List<Index> listIndexes(String dbName, String tblName, short max)
       throws NoSuchObjectException, MetaException, TException {
-    return client.get_indexes(dbName, tblName, max);
+    return filterHook.filterIndexes(client.get_indexes(dbName, tblName, max));
   }
 
   /** {@inheritDoc} */
@@ -1380,7 +1415,7 @@ public class HiveMetaStoreClient impleme
   @Override
   public Partition getPartition(String db, String tableName, String partName)
       throws MetaException, TException, UnknownTableException, NoSuchObjectException {
-    return deepCopy(client.get_partition_by_name(db, tableName, partName));
+    return deepCopy(filterHook.filterPartition(client.get_partition_by_name(db, tableName, partName)));
   }
 
   public Partition appendPartitionByName(String dbName, String tableName, String partName)