You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/07/31 02:43:28 UTC

[28/43] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
new file mode 100644
index 0000000..85e952f
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
@@ -0,0 +1,2863 @@
+/**
+ * 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.io.sarg;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.assertTrue;
+
+import com.google.common.collect.Sets;
+
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl.PredicateLeafImpl;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.junit.Test;
+
+import java.beans.XMLDecoder;
+import java.io.ByteArrayInputStream;
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.Field;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+
+/**
+ * These tests cover the conversion from Hive's AST to SearchArguments.
+ */
+public class TestConvertAstToSearchArg {
+
+  private static void assertNoSharedNodes(ExpressionTree tree,
+                                          Set<ExpressionTree> seen
+                                          ) throws Exception {
+    if (seen.contains(tree) &&
+        tree.getOperator() != ExpressionTree.Operator.LEAF) {
+      assertTrue("repeated node in expression " + tree, false);
+    }
+    seen.add(tree);
+    if (tree.getChildren() != null) {
+      for (ExpressionTree child : tree.getChildren()) {
+        assertNoSharedNodes(child, seen);
+      }
+    }
+  }
+
+  private ExprNodeGenericFuncDesc getFuncDesc(String xmlSerialized) {
+    byte[] bytes;
+    try {
+      bytes = xmlSerialized.getBytes("UTF-8");
+    } catch (UnsupportedEncodingException ex) {
+      throw new RuntimeException("UTF-8 support required", ex);
+    }
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+    XMLDecoder decoder = new XMLDecoder(bais, null, null);
+
+    try {
+      return (ExprNodeGenericFuncDesc) decoder.readObject();
+    } finally {
+      decoder.close();
+    }
+  }
+
+  @Test
+  public void testExpression1() throws Exception {
+    // first_name = 'john' or
+    //  'greg' < first_name or
+    //  'alan' > first_name or
+    //  id > 12 or
+    //  13 < id or
+    //  id < 15 or
+    //  16 > id or
+    //  (id <=> 30 and first_name <=> 'owen')
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                      <void property=\"children\"> \n" +
+        "                       <object class=\"java.util.ArrayList\"> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                          <void property=\"children\"> \n" +
+        "                           <object class=\"java.util.ArrayList\"> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                              <void property=\"children\"> \n" +
+        "                               <object class=\"java.util.ArrayList\"> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                                  <void property=\"column\"> \n" +
+        "                                   <string>first_name</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"tabAlias\"> \n" +
+        "                                   <string>orc_people</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                                    <void property=\"typeName\"> \n" +
+        "                                     <string>string</string> \n" +
+        "                                    </void> \n" +
+        "                                   </object> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"value\"> \n" +
+        "                                   <string>john</string> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                               </object> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"genericUDF\"> \n" +
+        "                               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                                <void property=\"typeName\"> \n" +
+        "                                 <string>boolean</string> \n" +
+        "                                </void> \n" +
+        "                               </object> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                              <void property=\"children\"> \n" +
+        "                               <object class=\"java.util.ArrayList\"> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"value\"> \n" +
+        "                                   <string>greg</string> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                                  <void property=\"column\"> \n" +
+        "                                   <string>first_name</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"tabAlias\"> \n" +
+        "                                   <string>orc_people</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                               </object> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"genericUDF\"> \n" +
+        "                               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                           </object> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"genericUDF\"> \n" +
+        "                           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                          <void property=\"children\"> \n" +
+        "                           <object class=\"java.util.ArrayList\"> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"value\"> \n" +
+        "                               <string>alan</string> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                              <void property=\"column\"> \n" +
+        "                               <string>first_name</string> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"tabAlias\"> \n" +
+        "                               <string>orc_people</string> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                           </object> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"genericUDF\"> \n" +
+        "                           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"genericUDF\"> \n" +
+        "                       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                      <void property=\"children\"> \n" +
+        "                       <object class=\"java.util.ArrayList\"> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                          <void property=\"column\"> \n" +
+        "                           <string>id</string> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"tabAlias\"> \n" +
+        "                           <string>orc_people</string> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                            <void property=\"typeName\"> \n" +
+        "                             <string>int</string> \n" +
+        "                            </void> \n" +
+        "                           </object> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"value\"> \n" +
+        "                           <int>12</int> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"genericUDF\"> \n" +
+        "                       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>13</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                      <void property=\"column\"> \n" +
+        "                       <string>id</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"tabAlias\"> \n" +
+        "                       <string>orc_people</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>id</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"value\"> \n" +
+        "                   <int>15</int> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>16</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>30</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>owen</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(9, leaves.size());
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String[] conditions = new String[]{
+      "eq(first_name, Binary{\"john\"})",    /* first_name = 'john' */
+      "not(lteq(first_name, Binary{\"greg\"}))", /* 'greg' < first_name */
+      "lt(first_name, Binary{\"alan\"})",   /* 'alan' > first_name */
+      "not(lteq(id, 12))",                  /* id > 12 or */
+      "not(lteq(id, 13))",                  /* 13 < id or */
+      "lt(id, 15)",                         /* id < 15 or */
+      "lt(id, 16)",                         /* 16 > id or */
+      "eq(id, 30)",                         /* id <=> 30 */
+      "eq(first_name, Binary{\"owen\"})"    /* first_name <=> 'owen' */
+    };
+    String expected = String
+      .format("and(or(or(or(or(or(or(or(%1$s, %2$s), %3$s), %4$s), %5$s), %6$s), %7$s), %8$s), " +
+        "or(or(or(or(or(or(or(%1$s, %2$s), %3$s), %4$s), %5$s), %6$s), %7$s), %9$s))", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("john", leaf.getLiteral());
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("greg", leaf.getLiteral());
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("alan", leaf.getLiteral());
+
+    leaf = leaves.get(3);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(12, leaf.getLiteral());
+
+    leaf = leaves.get(4);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(13, leaf.getLiteral());
+
+    leaf = leaves.get(5);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(15, leaf.getLiteral());
+
+    leaf = leaves.get(6);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(16, leaf.getLiteral());
+
+    leaf = leaves.get(7);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.NULL_SAFE_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(30, leaf.getLiteral());
+
+    leaf = leaves.get(8);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.NULL_SAFE_EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("owen", leaf.getLiteral());
+
+    assertEquals("(and (or leaf-0 (not leaf-1) leaf-2 (not leaf-3)" +
+        " (not leaf-4) leaf-5 leaf-6 leaf-7)" +
+        " (or leaf-0 (not leaf-1) leaf-2 (not leaf-3)" +
+        " (not leaf-4) leaf-5 leaf-6 leaf-8))",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+  }
+
+  @Test
+  public void testExpression2() throws Exception {
+    /* first_name is null or
+       first_name <> 'sue' or
+       id >= 12 or
+       id <= 4; */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>first_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                    <void property=\"typeName\"> \n" +
+        "                     <string>string</string> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>boolean</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>first_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"value\"> \n" +
+        "                   <string>sue</string> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>int</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>12</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "          <void property=\"column\"> \n" +
+        "           <string>id</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"tabAlias\"> \n" +
+        "           <string>orc_people</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <int>4</int> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(4, leaves.size());
+
+    String[] conditions = new String[]{
+      "eq(first_name, null)",               /* first_name is null  */
+      "not(eq(first_name, Binary{\"sue\"}))",    /* first_name <> 'sue' */
+      "not(lt(id, 12))",                    /* id >= 12            */
+      "lteq(id, 4)"                         /* id <= 4             */
+    };
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = String.format("or(or(or(%1$s, %2$s), %3$s), %4$s)", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.IS_NULL, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals(null, leaf.getLiteral());
+    assertEquals(null, leaf.getLiteralList());
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("sue", leaf.getLiteral());
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(12, leaf.getLiteral());
+
+    leaf = leaves.get(3);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(4, leaf.getLiteral());
+
+    assertEquals("(or leaf-0 (not leaf-1) (not leaf-2) leaf-3)",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.YES, TruthValue.YES, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NO, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
+            TruthValue.YES)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NULL, TruthValue.YES, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NULL,
+            TruthValue.NO)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
+            TruthValue.NULL)));
+    assertEquals(TruthValue.YES_NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NO, TruthValue.YES,
+            TruthValue.YES_NO)));
+    assertEquals(TruthValue.NO_NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NULL, TruthValue.YES,
+            TruthValue.NO_NULL)));
+    assertEquals(TruthValue.YES_NULL,
+        sarg.evaluate(values(TruthValue.YES_NULL, TruthValue.YES_NO_NULL,
+            TruthValue.YES, TruthValue.NULL)));
+    assertEquals(TruthValue.YES_NO_NULL,
+        sarg.evaluate(values(TruthValue.NO_NULL, TruthValue.YES_NO_NULL,
+            TruthValue.YES, TruthValue.NO)));
+  }
+
+  @Test
+  public void testExpression3() throws Exception {
+    /* (id between 23 and 45) and
+       first_name = 'alan' and
+       substr('xxxxx', 3) == first_name and
+       'smith' = last_name and
+       substr(first_name, 3) == 'yyy' */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                        <void property=\"typeName\"> \n" +
+        "                         <string>boolean</string> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <boolean>false</boolean> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                      <void property=\"column\"> \n" +
+        "                       <string>id</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"tabAlias\"> \n" +
+        "                       <string>orc_people</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                        <void property=\"typeName\"> \n" +
+        "                         <string>int</string> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>23</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>45</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                      <void property=\"column\"> \n" +
+        "                       <string>first_name</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"tabAlias\"> \n" +
+        "                       <string>orc_people</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                        <void property=\"typeName\"> \n" +
+        "                         <string>string</string> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <string>alan</string> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <string>xxxxx</string> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>3</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge\"> \n" +
+        "                    <void property=\"udfClassName\"> \n" +
+        "                     <string>org.apache.hadoop.hive.ql.udf.UDFSubstr</string> \n" +
+        "                    </void> \n" +
+        "                    <void property=\"udfName\"> \n" +
+        "                     <string>substr</string> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>first_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>smith</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>last_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>3</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge\"> \n" +
+        "            <void property=\"udfClassName\"> \n" +
+        "             <string>org.apache.hadoop.hive.ql.udf.UDFSubstr</string> \n" +
+        "            </void> \n" +
+        "            <void property=\"udfName\"> \n" +
+        "             <string>substr</string> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <string>yyy</string> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(3, leaves.size());
+
+    String[] conditions = new String[]{
+      "lt(id, 45)",                         /* id between 23 and 45 */
+      "not(lteq(id, 23))",                   /* id between 23 and 45 */
+      "eq(first_name, Binary{\"alan\"})",   /* first_name = 'alan'  */
+      "eq(last_name, Binary{\"smith\"})"    /* 'smith' = last_name  */
+    };
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = String.format("and(and(and(%1$s, %2$s), %3$s), %4$s)", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.BETWEEN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(null, leaf.getLiteral());
+    assertEquals(23, leaf.getLiteralList().get(0));
+    assertEquals(45, leaf.getLiteralList().get(1));
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("alan", leaf.getLiteral());
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("last_name", leaf.getColumnName());
+    assertEquals("smith", leaf.getLiteral());
+
+    assertEquals("(and leaf-0 leaf-1 leaf-2)",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+  }
+
+  @Test
+  public void testExpression4() throws Exception {
+    /* id <> 12 and
+       first_name in ('john', 'sue') and
+       id in (34,50) */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>int</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>12</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "            <void property=\"typeName\"> \n" +
+        "             <string>boolean</string> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>string</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>john</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>sue</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "          <void property=\"column\"> \n" +
+        "           <string>id</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"tabAlias\"> \n" +
+        "           <string>orc_people</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <int>34</int> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <int>50</int> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n" +
+        "\n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(3, leaves.size());
+
+    String[] conditions = new String[]{
+      "not(eq(id, 12))", /* id <> 12 */
+      "or(eq(first_name, Binary{\"john\"}), eq(first_name, Binary{\"sue\"}))", /* first_name in
+      ('john', 'sue') */
+      "or(eq(id, 34), eq(id, 50))" /* id in (34,50) */
+    };
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = String.format("and(and(%1$s, %2$s), %3$s)", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(12, leaf.getLiteral());
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.IN, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("john", leaf.getLiteralList().get(0));
+    assertEquals("sue", leaf.getLiteralList().get(1));
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.IN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(34, leaf.getLiteralList().get(0));
+    assertEquals(50, leaf.getLiteralList().get(1));
+
+    assertEquals("(and (not leaf-0) leaf-1 leaf-2)",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NULL, TruthValue.YES, TruthValue.YES)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES)));
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.YES, TruthValue.YES, TruthValue.YES)));
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO)));
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NULL, TruthValue.NO)));
+    assertEquals(TruthValue.NO_NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES_NO_NULL)));
+    assertEquals(TruthValue.NO_NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO_NULL)));
+  }
+
+  @Test
+  public void testExpression5() throws Exception {
+    /* (first_name < 'owen' or 'foobar' = substr(last_name, 4)) and
+    first_name between 'david' and 'greg' */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>string</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>owen</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "            <void property=\"typeName\"> \n" +
+        "             <string>boolean</string> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>foobar</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>last_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "          

<TRUNCATED>