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

hive git commit: HIVE-11208: Can not drop a default partition __HIVE_DEFAULT_PARTITION__ which is not a "string" type (Aihua Xu, reviewed by Yongzhi Chen)

Repository: hive
Updated Branches:
  refs/heads/master 085650e0c -> 2de8b3192


HIVE-11208: Can not drop a default partition __HIVE_DEFAULT_PARTITION__ which is not a "string" type (Aihua Xu, reviewed by Yongzhi Chen)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2de8b319
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2de8b319
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2de8b319

Branch: refs/heads/master
Commit: 2de8b3192306376a382f6b212f24fde3f990e57d
Parents: 085650e
Author: Aihua Xu <ai...@apache.org>
Authored: Fri Mar 4 17:03:18 2016 -0500
Committer: Aihua Xu <ai...@apache.org>
Committed: Thu Nov 17 12:00:27 2016 -0500

----------------------------------------------------------------------
 .../exec/ExprNodeConstantDefaultEvaluator.java  |  50 ++++++
 .../hive/ql/exec/ExprNodeEvaluatorFactory.java  |   7 +
 .../ql/optimizer/ppr/PartExprEvalUtils.java     |   8 +-
 .../hive/ql/optimizer/ppr/PartitionPruner.java  |  19 ++-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  23 ++-
 .../ql/plan/ExprNodeConstantDefaultDesc.java    |  86 ++++++++++
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |   9 +
 .../ql/udf/generic/GenericUDFOPNotEqual.java    |   9 +
 .../clientpositive/drop_partitions_filter4.q    |  29 ++++
 .../drop_partitions_filter4.q.out               | 167 +++++++++++++++++++
 10 files changed, 390 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java
new file mode 100644
index 0000000..89a75eb
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeConstantDefaultEvaluator.java
@@ -0,0 +1,50 @@
+/**
+ * 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.ql.exec;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+/**
+ * ExprNodeConstantEvaluator.
+ *
+ */
+public class ExprNodeConstantDefaultEvaluator extends ExprNodeEvaluator<ExprNodeConstantDefaultDesc> {
+
+  transient ObjectInspector writableObjectInspector;
+
+  public ExprNodeConstantDefaultEvaluator(ExprNodeConstantDefaultDesc expr) {
+    super(expr);
+    writableObjectInspector = expr.getWritableObjectInspector();
+  }
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector rowInspector) throws HiveException {
+    return writableObjectInspector;
+  }
+
+  @Override
+  protected Object _evaluate(Object row, int version) throws HiveException {
+    return expr;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
index 5a532c4..0d03d8f 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeEvaluatorFactory.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
@@ -42,6 +43,12 @@ public final class ExprNodeEvaluatorFactory {
     if (desc instanceof ExprNodeConstantDesc) {
       return new ExprNodeConstantEvaluator((ExprNodeConstantDesc) desc);
     }
+
+    // Special 'default' constant node
+    if (desc instanceof ExprNodeConstantDefaultDesc) {
+      return new ExprNodeConstantDefaultEvaluator((ExprNodeConstantDefaultDesc) desc);
+    }
+
     // Column-reference node, e.g. a column in the input row
     if (desc instanceof ExprNodeColumnDesc) {
       return new ExprNodeColumnEvaluator((ExprNodeColumnDesc) desc);

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
index dc5d2df..1103d35 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
@@ -104,16 +104,16 @@ public class PartExprEvalUtils {
   }
 
   static synchronized public ObjectPair<PrimitiveObjectInspector, ExprNodeEvaluator> prepareExpr(
-      ExprNodeGenericFuncDesc expr, List<String> partNames,
+      ExprNodeGenericFuncDesc expr, List<String> partColumnNames,
       List<PrimitiveTypeInfo> partColumnTypeInfos) throws HiveException {
     // Create the row object
     List<ObjectInspector> partObjectInspectors = new ArrayList<ObjectInspector>();
-    for (int i = 0; i < partNames.size(); i++) {
+    for (int i = 0; i < partColumnNames.size(); i++) {
       partObjectInspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(
-          partColumnTypeInfos.get(i)));
+        partColumnTypeInfos.get(i)));
     }
     StructObjectInspector objectInspector = ObjectInspectorFactory
-        .getStandardStructObjectInspector(partNames, partObjectInspectors);
+        .getStandardStructObjectInspector(partColumnNames, partObjectInspectors);
 
     ExprNodeEvaluator evaluator = ExprNodeEvaluatorFactory.get(expr);
     ObjectInspector evaluateResultOI = evaluator.initialize(objectInspector);

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
index 26e936e..9d7307e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
@@ -62,6 +63,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -542,11 +544,18 @@ public class PartitionPruner extends Transform {
 
       ArrayList<Object> convertedValues = new ArrayList<Object>(values.size());
       for(int i=0; i<values.size(); i++) {
-        Object o = ObjectInspectorConverters.getConverter(
-            PrimitiveObjectInspectorFactory.javaStringObjectInspector,
-            PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(partColumnTypeInfos.get(i)))
-            .convert(values.get(i));
-        convertedValues.add(o);
+        String partitionValue = values.get(i);
+        PrimitiveTypeInfo typeInfo = partColumnTypeInfos.get(i);
+
+        if (partitionValue.equals(defaultPartitionName)) {
+          convertedValues.add(new ExprNodeConstantDefaultDesc(typeInfo, defaultPartitionName));
+        } else {
+          Object o = ObjectInspectorConverters.getConverter(
+              PrimitiveObjectInspectorFactory.javaStringObjectInspector,
+              PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(typeInfo))
+              .convert(partitionValue);
+          convertedValues.add(o);
+        }
       }
 
       // Evaluate the expression tree.

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index a264c4d..c7389a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -101,6 +101,7 @@ import org.apache.hadoop.hive.ql.plan.DropIndexDesc;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
@@ -3093,6 +3094,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   private Map<Integer, List<ExprNodeGenericFuncDesc>> getFullPartitionSpecs(
       CommonTree ast, Table tab, boolean canGroupExprs) throws SemanticException {
+    String defaultPartitionName = HiveConf.getVar(conf, HiveConf.ConfVars.DEFAULTPARTITIONNAME);
     Map<String, String> colTypes = new HashMap<String, String>();
     for (FieldSchema fs : tab.getPartitionKeys()) {
       colTypes.put(fs.getName().toLowerCase(), fs.getType());
@@ -3114,23 +3116,28 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         TypeCheckCtx typeCheckCtx = new TypeCheckCtx(null);
         ExprNodeConstantDesc valExpr = (ExprNodeConstantDesc)TypeCheckProcFactory
             .genExprNode(partValNode, typeCheckCtx).get(partValNode);
+        Object val = valExpr.getValue();
+
+        boolean isDefaultPartitionName =  val.equals(defaultPartitionName);
 
         String type = colTypes.get(key);
+        PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(type);
         if (type == null) {
           throw new SemanticException("Column " + key + " not found");
         }
         // Create the corresponding hive expression to filter on partition columns.
-        PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(type);
-        Object val = valExpr.getValue();
-        if (!valExpr.getTypeString().equals(type)) {
-          Converter converter = ObjectInspectorConverters.getConverter(
-            TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(valExpr.getTypeInfo()),
-            TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(pti));
-          val = converter.convert(valExpr.getValue());
+        if (!isDefaultPartitionName) {
+          if (!valExpr.getTypeString().equals(type)) {
+            Converter converter = ObjectInspectorConverters.getConverter(
+              TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(valExpr.getTypeInfo()),
+              TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(pti));
+            val = converter.convert(valExpr.getValue());
+          }
         }
+
         ExprNodeColumnDesc column = new ExprNodeColumnDesc(pti, key, null, true);
         ExprNodeGenericFuncDesc op = makeBinaryPredicate(operator, column,
-            new ExprNodeConstantDesc(pti, val));
+            isDefaultPartitionName ? new ExprNodeConstantDefaultDesc(pti, defaultPartitionName) : new ExprNodeConstantDesc(pti, val));
         // If it's multi-expr filter (e.g. a='5', b='2012-01-02'), AND with previous exprs.
         expr = (expr == null) ? op : makeBinaryPredicate("and", expr, op);
         names.add(key);

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java
new file mode 100644
index 0000000..8b3c0bc
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDefaultDesc.java
@@ -0,0 +1,86 @@
+/**
+ * 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.ql.plan;
+
+import java.io.Serializable;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * A constant expression with default value and data type. The value is different
+ * from any value of that data type. Used to represent the default partition in
+ * the expression of x =/!= __HIVE_DEFAULT_PARTITION__
+ */
+public class ExprNodeConstantDefaultDesc extends ExprNodeDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private final Object value;     // The internal value for the default
+
+  public ExprNodeConstantDefaultDesc() {
+    value = null;
+  }
+
+  public ExprNodeConstantDefaultDesc(TypeInfo typeInfo, Object value) {
+    super(typeInfo);
+    this.value = value;
+  }
+
+  @Override
+  public String toString() {
+    return "Const " + typeInfo.toString() + " default";
+  }
+
+  @Override
+  public String getExprString() {
+    return value == null ? "null" : value.toString();
+  }
+
+  @Override
+  public boolean isSame(Object o) {
+    if (!(o instanceof ExprNodeConstantDefaultDesc)) {
+      return false;
+    }
+    ExprNodeConstantDefaultDesc dest = (ExprNodeConstantDefaultDesc) o;
+    if (!typeInfo.equals(dest.getTypeInfo())) {
+      return false;
+    }
+    if (value == null) {
+      if (dest.value != null) {
+        return false;
+      }
+    } else if (!value.equals(dest.value)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public ExprNodeDesc clone() {
+    return new ExprNodeConstantDefaultDesc(typeInfo, value);
+  }
+
+  @Override
+  public int hashCode() {
+    int superHashCode = super.hashCode();
+    HashCodeBuilder builder = new HashCodeBuilder();
+    builder.appendSuper(superHashCode);
+    return builder.toHashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
index b393843..0002b07 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 
 /**
@@ -109,6 +110,14 @@ public class GenericUDFOPEqual extends GenericUDFBaseCompare {
       return null;
     }
 
+    // Handle 'default' constant which has a data type with special value
+    if (o0 instanceof ExprNodeConstantDefaultDesc || o1 instanceof ExprNodeConstantDefaultDesc) {
+      ExprNodeConstantDefaultDesc default0 = o0 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o0 : null;
+      ExprNodeConstantDefaultDesc default1 = o1 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o1 : null;
+      result.set(default0 != null && default1 != null && default0.isSame(default1));
+      return result;
+    }
+
     switch(compareType) {
     case COMPARE_TEXT:
       result.set(soi0.getPrimitiveWritableObject(o0).equals(

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
index ed6aa36..9652859 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScal
 import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDefaultDesc;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 
 /**
@@ -109,6 +110,14 @@ public class GenericUDFOPNotEqual extends GenericUDFBaseCompare {
       return null;
     }
 
+    // Handle 'default' constant which has a data type with special value
+    if (o0 instanceof ExprNodeConstantDefaultDesc || o1 instanceof ExprNodeConstantDefaultDesc) {
+      ExprNodeConstantDefaultDesc default0 = o0 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o0 : null;
+      ExprNodeConstantDefaultDesc default1 = o1 instanceof ExprNodeConstantDefaultDesc ? (ExprNodeConstantDefaultDesc)o1 : null;
+      result.set(default0 == null || default1 == null || !default0.isSame(default1));
+      return result;
+    }
+
     switch(compareType) {
     case COMPARE_TEXT:
       result.set(!soi0.getPrimitiveWritableObject(o0).equals(

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/test/queries/clientpositive/drop_partitions_filter4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/drop_partitions_filter4.q b/ql/src/test/queries/clientpositive/drop_partitions_filter4.q
new file mode 100644
index 0000000..f0e5e19
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/drop_partitions_filter4.q
@@ -0,0 +1,29 @@
+SET hive.exec.dynamic.partition.mode=nonstrict;
+
+create table ptestfilter (a string, b int) partitioned by (c double);
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c) select 'Col1', 1, null;
+alter table ptestfilter add partition (c=3.4);
+alter table ptestfilter add partition (c=5.55);
+show partitions ptestfilter;
+
+alter table ptestfilter drop partition(c = '__HIVE_DEFAULT_PARTITION__');
+alter table ptestfilter drop partition(c = 3.40);
+show partitions ptestfilter;
+
+drop table ptestfilter;
+
+create table ptestfilter (a string, b int) partitioned by (c string, d int);
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col1', 1, null, null;
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col2', 2, null, 2;
+INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col3', 3, 'Uganda', null;
+alter table ptestfilter add partition (c='Germany', d=2);
+show partitions ptestfilter;
+
+alter table ptestfilter drop partition (c='__HIVE_DEFAULT_PARTITION__');
+alter table ptestfilter drop partition (c='Uganda', d='__HIVE_DEFAULT_PARTITION__');
+alter table ptestfilter drop partition (c='Germany', d=2);
+show partitions ptestfilter;
+
+drop table ptestfilter;
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/2de8b319/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out b/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out
new file mode 100644
index 0000000..0e6d41a
--- /dev/null
+++ b/ql/src/test/results/clientpositive/drop_partitions_filter4.q.out
@@ -0,0 +1,167 @@
+PREHOOK: query: create table ptestfilter (a string, b int) partitioned by (c double)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: create table ptestfilter (a string, b int) partitioned by (c double)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ptestfilter
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c) select 'Col1', 1, null
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c) select 'Col1', 1, null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
+PREHOOK: query: alter table ptestfilter add partition (c=3.4)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: alter table ptestfilter add partition (c=3.4)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=3.4
+PREHOOK: query: alter table ptestfilter add partition (c=5.55)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: alter table ptestfilter add partition (c=5.55)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=5.55
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+c=3.4
+c=5.55
+c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition(c = '__HIVE_DEFAULT_PARTITION__')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: query: alter table ptestfilter drop partition(c = '__HIVE_DEFAULT_PARTITION__')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition(c = 3.40)
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=3.4
+POSTHOOK: query: alter table ptestfilter drop partition(c = 3.40)
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=3.4
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+c=5.55
+PREHOOK: query: drop table ptestfilter
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: drop table ptestfilter
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter
+PREHOOK: query: create table ptestfilter (a string, b int) partitioned by (c string, d int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: create table ptestfilter (a string, b int) partitioned by (c string, d int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ptestfilter
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col1', 1, null, null
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col1', 1, null, null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col2', 2, null, 2
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col2', 2, null, 2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=2
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=2).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=__HIVE_DEFAULT_PARTITION__,d=2).b SIMPLE []
+PREHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col3', 3, 'Uganda', null
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: INSERT OVERWRITE TABLE ptestfilter PARTITION (c,d) select 'Col3', 3, 'Uganda', null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@ptestfilter@c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: Lineage: ptestfilter PARTITION(c=Uganda,d=__HIVE_DEFAULT_PARTITION__).a SIMPLE []
+POSTHOOK: Lineage: ptestfilter PARTITION(c=Uganda,d=__HIVE_DEFAULT_PARTITION__).b SIMPLE []
+PREHOOK: query: alter table ptestfilter add partition (c='Germany', d=2)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: alter table ptestfilter add partition (c='Germany', d=2)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=Germany/d=2
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+c=Germany/d=2
+c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+c=__HIVE_DEFAULT_PARTITION__/d=2
+c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition (c='__HIVE_DEFAULT_PARTITION__')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=2
+PREHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: query: alter table ptestfilter drop partition (c='__HIVE_DEFAULT_PARTITION__')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=2
+POSTHOOK: Output: default@ptestfilter@c=__HIVE_DEFAULT_PARTITION__/d=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition (c='Uganda', d='__HIVE_DEFAULT_PARTITION__')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+POSTHOOK: query: alter table ptestfilter drop partition (c='Uganda', d='__HIVE_DEFAULT_PARTITION__')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=Uganda/d=__HIVE_DEFAULT_PARTITION__
+PREHOOK: query: alter table ptestfilter drop partition (c='Germany', d=2)
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter@c=Germany/d=2
+POSTHOOK: query: alter table ptestfilter drop partition (c='Germany', d=2)
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter@c=Germany/d=2
+PREHOOK: query: show partitions ptestfilter
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: default@ptestfilter
+POSTHOOK: query: show partitions ptestfilter
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: default@ptestfilter
+PREHOOK: query: drop table ptestfilter
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@ptestfilter
+PREHOOK: Output: default@ptestfilter
+POSTHOOK: query: drop table ptestfilter
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@ptestfilter
+POSTHOOK: Output: default@ptestfilter