You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2017/07/14 17:41:41 UTC

hive git commit: HIVE-16989 : Fix some issues identified by lgtm.com (Malcolm Taylor via Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master adca35a46 -> 094c1d503


HIVE-16989 : Fix some issues identified by lgtm.com (Malcolm Taylor via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: 094c1d503d4845ec9613d08c7e6fb262e3fc38a9
Parents: adca35a
Author: Malcolm Taylor <ma...@semmle.com>
Authored: Fri Jun 30 16:24:00 2017 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Jul 14 10:41:22 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |   3 +-
 .../impl/StaticPermanentFunctionChecker.java    |   3 +-
 .../hive/metastore/MetaStoreSchemaInfo.java     |   4 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  96 +++++++++-------
 .../hadoop/hive/ql/exec/KeyWrapperFactory.java  |   6 +
 .../ql/exec/persistence/FlatRowContainer.java   |   2 +-
 .../hive/ql/io/parquet/convert/Repeated.java    |   1 -
 .../hive/ql/parse/repl/load/DumpMetaData.java   |   5 +-
 .../hadoop/hive/ql/plan/TezEdgeProperty.java    |   2 +-
 .../GenericUDAFMkCollectionEvaluator.java       |   2 +-
 .../hive/ql/exec/TestKeyWrapperFactory.java     | 113 +++++++++++++++++++
 11 files changed, 181 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 33a65d4..fcc0cd5 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -2182,8 +2182,7 @@ public class BeeLine implements Closeable {
       output(getColorBuffer().pad(loc("scanning", f.getAbsolutePath()), 60),
           false);
 
-      try {
-        ZipFile zf = new ZipFile(f);
+      try (ZipFile zf = new ZipFile(f)) {
         int total = zf.size();
         int index = 0;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StaticPermanentFunctionChecker.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StaticPermanentFunctionChecker.java b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StaticPermanentFunctionChecker.java
index 15968fa..1d2565f 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StaticPermanentFunctionChecker.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/StaticPermanentFunctionChecker.java
@@ -43,8 +43,7 @@ public class StaticPermanentFunctionChecker implements UdfWhitelistChecker {
       LOG.warn("Could not find UDF whitelist in configuration: " + PERMANENT_FUNCTIONS_LIST);
       return;
     }
-    try {
-      BufferedReader r = new BufferedReader(new InputStreamReader(logger.openStream()));
+    try (BufferedReader r = new BufferedReader(new InputStreamReader(logger.openStream()))) {
       String klassName = r.readLine();
       while (klassName != null) {
         try {

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
index d6d83e0..8117ca1 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreSchemaInfo.java
@@ -216,8 +216,8 @@ public class MetaStoreSchemaInfo implements IMetaStoreSchemaInfo {
       versionQuery = "select t.SCHEMA_VERSION from VERSION t";
     }
     try (Connection metastoreDbConnection =
-        HiveSchemaHelper.getConnectionToMetastore(connectionInfo)) {
-      Statement stmt = metastoreDbConnection.createStatement();
+        HiveSchemaHelper.getConnectionToMetastore(connectionInfo); Statement stmt =
+        metastoreDbConnection.createStatement()) {
       ResultSet res = stmt.executeQuery(versionQuery);
       if (!res.next()) {
         throw new HiveMetaException("Could not find version info in metastore VERSION table.");

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 970038d..b722af6 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -1484,16 +1484,17 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     // Get the id for the next entry in the queue
     String s = sqlGenerator.addForUpdateClause("select ncq_next from NEXT_COMPACTION_QUEUE_ID");
     LOG.debug("going to execute query <" + s + ">");
-    ResultSet rs = stmt.executeQuery(s);
-    if (!rs.next()) {
-      throw new IllegalStateException("Transaction tables not properly initiated, " +
-        "no record found in next_compaction_queue_id");
+    try (ResultSet rs = stmt.executeQuery(s)) {
+      if (!rs.next()) {
+        throw new IllegalStateException("Transaction tables not properly initiated, "
+            + "no record found in next_compaction_queue_id");
+      }
+      long id = rs.getLong(1);
+      s = "update NEXT_COMPACTION_QUEUE_ID set ncq_next = " + (id + 1);
+      LOG.debug("Going to execute update <" + s + ">");
+      stmt.executeUpdate(s);
+      return id;
     }
-    long id = rs.getLong(1);
-    s = "update NEXT_COMPACTION_QUEUE_ID set ncq_next = " + (id + 1);
-    LOG.debug("Going to execute update <" + s + ">");
-    stmt.executeUpdate(s);
-    return id;
   }
   @Override
   @RetrySemantics.Idempotent
@@ -2863,22 +2864,26 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   private TxnStatus findTxnState(long txnid, Statement stmt) throws SQLException, MetaException {
     String s = "select txn_state from TXNS where txn_id = " + txnid;
     LOG.debug("Going to execute query <" + s + ">");
-    ResultSet rs = stmt.executeQuery(s);
-    if (!rs.next()) {
-      s = sqlGenerator.addLimitClause(1, "1 from COMPLETED_TXN_COMPONENTS where CTC_TXNID = " + txnid);
-      LOG.debug("Going to execute query <" + s + ">");
-      ResultSet rs2 = stmt.executeQuery(s);
-      if(rs2.next()) {
-        return TxnStatus.COMMITTED;
+    try (ResultSet rs = stmt.executeQuery(s)) {
+      if (!rs.next()) {
+        s =
+            sqlGenerator.addLimitClause(1, "1 from COMPLETED_TXN_COMPONENTS where CTC_TXNID = "
+                + txnid);
+        LOG.debug("Going to execute query <" + s + ">");
+        try (ResultSet rs2 = stmt.executeQuery(s)) {
+          if (rs2.next()) {
+            return TxnStatus.COMMITTED;
+          }
+        }
+        // could also check WRITE_SET but that seems overkill
+        return TxnStatus.UNKNOWN;
       }
-      //could also check WRITE_SET but that seems overkill
-      return TxnStatus.UNKNOWN;
-    }
-    char txnState = rs.getString(1).charAt(0);
-    if (txnState == TXN_ABORTED) {
-      return TxnStatus.ABORTED;
+      char txnState = rs.getString(1).charAt(0);
+      if (txnState == TXN_ABORTED) {
+        return TxnStatus.ABORTED;
+      }
+      assert txnState == TXN_OPEN : "we found it in TXNS but it's not ABORTED, so must be OPEN";
     }
-    assert txnState == TXN_OPEN : "we found it in TXNS but it's not ABORTED, so must be OPEN";
     return TxnStatus.OPEN;
   }
 
@@ -2909,27 +2914,32 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     // We need to check whether this transaction is valid and open
     String s = "select txn_state from TXNS where txn_id = " + txnid;
     LOG.debug("Going to execute query <" + s + ">");
-    ResultSet rs = stmt.executeQuery(s);
-    if (!rs.next()) {
-      //todo: add LIMIT 1 instead of count - should be more efficient
-      s = "select count(*) from COMPLETED_TXN_COMPONENTS where CTC_TXNID = " + txnid;
-      ResultSet rs2 = stmt.executeQuery(s);
-      //todo: strictly speaking you can commit an empty txn, thus 2nd conjunct is wrong but only
-      //possible for for multi-stmt txns
-      boolean alreadyCommitted = rs2.next() && rs2.getInt(1) > 0;
-      LOG.debug("Going to rollback");
-      rollbackDBConn(dbConn);
-      if(alreadyCommitted) {
-        //makes the message more informative - helps to find bugs in client code
-        throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid) + " is already committed.");
+    try (ResultSet rs = stmt.executeQuery(s)) {
+      if (!rs.next()) {
+        // todo: add LIMIT 1 instead of count - should be more efficient
+        s = "select count(*) from COMPLETED_TXN_COMPONENTS where CTC_TXNID = " + txnid;
+        try (ResultSet rs2 = stmt.executeQuery(s)) {
+          // todo: strictly speaking you can commit an empty txn, thus 2nd conjunct is wrong but
+          // only
+          // possible for for multi-stmt txns
+          boolean alreadyCommitted = rs2.next() && rs2.getInt(1) > 0;
+          LOG.debug("Going to rollback");
+          rollbackDBConn(dbConn);
+          if (alreadyCommitted) {
+            // makes the message more informative - helps to find bugs in client code
+            throw new NoSuchTxnException("Transaction " + JavaUtils.txnIdToString(txnid)
+                + " is already committed.");
+          }
+          throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid));
+        }
+      }
+      if (rs.getString(1).charAt(0) == TXN_ABORTED) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid)
+            + " already aborted");// todo: add time of abort, which is not currently tracked.
+                                  // Requires schema change
       }
-      throw new NoSuchTxnException("No such transaction " + JavaUtils.txnIdToString(txnid));
-    }
-    if (rs.getString(1).charAt(0) == TXN_ABORTED) {
-      LOG.debug("Going to rollback");
-      rollbackDBConn(dbConn);
-      throw new TxnAbortedException("Transaction " + JavaUtils.txnIdToString(txnid) +
-        " already aborted");//todo: add time of abort, which is not currently tracked.  Requires schema change
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
index 5154a5f..775c737 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/KeyWrapperFactory.java
@@ -99,6 +99,9 @@ public class KeyWrapperFactory {
 
     @Override
     public boolean equals(Object obj) {
+      if (!(obj instanceof ListKeyWrapper)) {
+        return false;
+      }
       Object[] copied_in_hashmap = ((ListKeyWrapper) obj).keys;
       return equalComparer.areEqual(copied_in_hashmap, keys);
     }
@@ -182,6 +185,9 @@ public class KeyWrapperFactory {
 
     @Override
     public boolean equals(Object other) {
+      if (!(other instanceof TextKeyWrapper)) {
+        return false;
+      }
       Object obj = ((TextKeyWrapper) other).key;
       Text t1;
       Text t2;

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/FlatRowContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/FlatRowContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/FlatRowContainer.java
index 9b1af1b..48d7a8a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/FlatRowContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/FlatRowContainer.java
@@ -292,7 +292,7 @@ public class FlatRowContainer extends AbstractCollection<Object>
     }
 
     public Iterator<Object> iterator() {
-      return listIterator();
+      return super.listIterator();
     }
 
     public ListIterator<Object> listIterator(int index) {

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
index a7fad71..a302f16 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/convert/Repeated.java
@@ -147,7 +147,6 @@ public interface Repeated extends ConverterParent {
     private final ConverterParent parent;
     private final int index;
     private final List<Writable> list = new ArrayList<Writable>();
-    private final Map<String, String> metadata = new HashMap<String, String>();
 
 
     public RepeatedGroupConverter(GroupType groupType, ConverterParent parent, int index, TypeInfo hiveTypeInfo) {

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
index 2d5e3b1..f12bd13 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/DumpMetaData.java
@@ -66,10 +66,9 @@ public class DumpMetaData {
   }
 
   private void loadDumpFromFile() throws SemanticException {
-    try {
+    try (FileSystem fs = dumpFile.getFileSystem(hiveConf); BufferedReader br =
+        new BufferedReader(new InputStreamReader(fs.open(dumpFile)))) {
       // read from dumpfile and instantiate self
-      FileSystem fs = dumpFile.getFileSystem(hiveConf);
-      BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(dumpFile)));
       String line = null;
       if ((line = br.readLine()) != null) {
         String[] lineContents = line.split("\t", 5);

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/java/org/apache/hadoop/hive/ql/plan/TezEdgeProperty.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TezEdgeProperty.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezEdgeProperty.java
index 5d7ddc8..a55c708 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TezEdgeProperty.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TezEdgeProperty.java
@@ -51,6 +51,7 @@ public class TezEdgeProperty {
       boolean isSlowStart, int minReducer, int maxReducer, long bytesPerReducer) {
     this(hiveConf, edgeType, -1);
     setAutoReduce(hiveConf, isAutoReduce, minReducer, maxReducer, bytesPerReducer);
+    this.isSlowStart = isSlowStart;
   }
 
   public void setAutoReduce(HiveConf hiveConf, boolean isAutoReduce, int minReducer,
@@ -60,7 +61,6 @@ public class TezEdgeProperty {
     this.maxReducer = maxReducer;
     this.isAutoReduce = isAutoReduce;
     this.inputSizePerReducer = bytesPerReducer;
-    this.isSlowStart = isSlowStart;
   }
 
   public TezEdgeProperty(EdgeType edgeType) {

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMkCollectionEvaluator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMkCollectionEvaluator.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMkCollectionEvaluator.java
index 2b5e6dd..e2468b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMkCollectionEvaluator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFMkCollectionEvaluator.java
@@ -134,7 +134,7 @@ public class GenericUDAFMkCollectionEvaluator extends GenericUDAFEvaluator
   public void merge(AggregationBuffer agg, Object partial)
       throws HiveException {
     MkArrayAggregationBuffer myagg = (MkArrayAggregationBuffer) agg;
-    List<Object> partialResult = (ArrayList<Object>) internalMergeOI.getList(partial);
+    List<Object> partialResult = (List<Object>) internalMergeOI.getList(partial);
     if (partialResult != null) {
       for(Object i : partialResult) {
         putIntoCollection(i, myagg);

http://git-wip-us.apache.org/repos/asf/hive/blob/094c1d50/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java
new file mode 100644
index 0000000..119959c
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestKeyWrapperFactory.java
@@ -0,0 +1,113 @@
+/**
+ * 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 static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.KeyWrapperFactory.ListKeyWrapper;
+import org.apache.hadoop.hive.ql.exec.KeyWrapperFactory.TextKeyWrapper;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestKeyWrapperFactory {
+  private KeyWrapperFactory factory;
+
+
+  @Before
+  public void setup() throws Exception {
+    SessionState ss = new SessionState(new HiveConf());
+    SessionState.setCurrentSessionState(ss);
+
+    ArrayList<Text> col1 = new ArrayList<Text>();
+    col1.add(new Text("0"));
+    col1.add(new Text("1"));
+    col1.add(new Text("2"));
+    col1.add(new Text("3"));
+    TypeInfo col1Type = TypeInfoFactory.getListTypeInfo(TypeInfoFactory.stringTypeInfo);
+    ArrayList<Text> cola = new ArrayList<Text>();
+    cola.add(new Text("a"));
+    cola.add(new Text("b"));
+    cola.add(new Text("c"));
+    TypeInfo colaType = TypeInfoFactory.getListTypeInfo(TypeInfoFactory.stringTypeInfo);
+    try {
+      ArrayList<Object> data = new ArrayList<Object>();
+      data.add(col1);
+      data.add(cola);
+      ArrayList<String> names = new ArrayList<String>();
+      names.add("col1");
+      names.add("cola");
+      ArrayList<TypeInfo> typeInfos = new ArrayList<TypeInfo>();
+      typeInfos.add(col1Type);
+      typeInfos.add(colaType);
+      TypeInfo dataType = TypeInfoFactory.getStructTypeInfo(names, typeInfos);
+
+      InspectableObject r = new InspectableObject();
+      ObjectInspector[] oi = new ObjectInspector[1];
+      r.o = data;
+      oi[0]= TypeInfoUtils
+          .getStandardWritableObjectInspectorFromTypeInfo(dataType);
+      try {
+        // get a evaluator for a simple field expression
+        ExprNodeDesc exprDesc = new ExprNodeColumnDesc(colaType, "cola", "",
+            false);
+        ExprNodeEvaluator eval = ExprNodeEvaluatorFactory.get(exprDesc);
+        ExprNodeEvaluator[] evals = new ExprNodeEvaluator[1];
+        evals[0] = eval;
+        ObjectInspector resultOI = eval.initialize(oi[0]);
+        ObjectInspector[] resultOIs = new ObjectInspector[1];
+        resultOIs[0] = resultOI;
+        factory = new KeyWrapperFactory(evals, oi, resultOIs);
+      } catch (Throwable e) {
+        e.printStackTrace();
+        throw e;
+      }
+    } catch (Throwable e) {
+      e.printStackTrace();
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Test
+  public void testKeyWrapperEqualsCopy() throws Exception {
+    KeyWrapper w1 = factory.getKeyWrapper();
+    KeyWrapper w2 = w1.copyKey();
+    assertTrue(w1.equals(w2));
+  }
+
+  @Test
+  public void testDifferentWrapperTypesUnequal() {
+    TextKeyWrapper w3 = factory.new TextKeyWrapper(false);
+    ListKeyWrapper w4 = factory.new ListKeyWrapper(false);
+    assertFalse(w3.equals(w4));
+    assertFalse(w4.equals(w3));
+  }
+}