This is an automated email from the ASF dual-hosted git repository.

mgergely pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 029cab2  HIVE-22192 Remove ObjectPair classes
029cab2 is described below

commit 029cab297a9ae40d249f63040721f93857398648
Author: miklosgergely <mgerg...@cloudera.com>
AuthorDate: Wed Sep 11 10:33:46 2019 +0200

    HIVE-22192 Remove ObjectPair classes
---
 .../org/apache/hadoop/hive/common/ObjectPair.java  | 86 ----------------------
 .../hive/hcatalog/api/HCatClientHMSImpl.java       |  4 +-
 .../AlterTableDropPartitionOperation.java          |  6 +-
 .../apache/hadoop/hive/ql/exec/FooterBuffer.java   | 18 +++--
 .../hadoop/hive/ql/exec/MapJoinOperator.java       | 14 +---
 .../hadoop/hive/ql/exec/SMBMapJoinOperator.java    | 17 +++--
 .../ql/exec/persistence/KeyValueContainer.java     | 18 ++---
 .../hive/ql/exec/spark/HiveKVResultCache.java      | 56 +++++++-------
 .../hive/ql/exec/spark/session/SparkSession.java   |  4 +-
 .../ql/exec/spark/session/SparkSessionImpl.java    |  9 +--
 .../apache/hadoop/hive/ql/hooks/LineageLogger.java | 12 +--
 .../org/apache/hadoop/hive/ql/metadata/Hive.java   | 76 ++++++++++++++++---
 .../hive/ql/optimizer/ConvertJoinMapJoin.java      |  2 +-
 .../hadoop/hive/ql/optimizer/MapJoinProcessor.java | 16 ++--
 .../hive/ql/optimizer/lineage/LineageCtx.java      | 20 ++---
 .../hive/ql/optimizer/ppr/PartExprEvalUtils.java   | 10 +--
 .../hive/ql/optimizer/ppr/PartitionPruner.java     |  4 +-
 .../spark/SetSparkReducerParallelism.java          | 10 +--
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |  2 +-
 .../apache/hadoop/hive/ql/parse/QBSubQuery.java    | 32 ++++----
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     | 65 ++++++++--------
 .../hive/ql/parse/spark/GenSparkProcContext.java   |  8 +-
 .../hadoop/hive/ql/parse/spark/GenSparkWork.java   | 13 ++--
 .../TestReflectionObjectInspectors.java            | 17 ++---
 .../hadoop/hive/metastore/HiveMetaStoreClient.java | 19 ++---
 .../hadoop/hive/metastore/IMetaStoreClient.java    | 14 ++--
 .../hadoop/hive/metastore/utils/ObjectPair.java    | 86 ----------------------
 .../org/apache/hadoop/hive/metastore/Msck.java     | 10 +--
 .../apache/hadoop/hive/metastore/ObjectStore.java  | 75 ++++++++-----------
 .../metastore/HiveMetaStoreClientPreCatalog.java   | 19 +++--
 30 files changed, 290 insertions(+), 452 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/common/ObjectPair.java 
b/common/src/java/org/apache/hadoop/hive/common/ObjectPair.java
deleted file mode 100644
index 50b5b3f..0000000
--- a/common/src/java/org/apache/hadoop/hive/common/ObjectPair.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.common;
-
-
-
-public class ObjectPair<F, S> {
-  private F first;
-  private S second;
-
-  public ObjectPair() {}
-
-  /**
-   * Creates a pair. Constructor doesn't infer template args but
-   * the method does, so the code becomes less ugly.
-   */
-  public static <T1, T2> ObjectPair<T1, T2> create(T1 f, T2 s) {
-    return new ObjectPair<T1, T2>(f, s);
-  }
-
-  public ObjectPair(F first, S second) {
-    this.first = first;
-    this.second = second;
-  }
-
-  public F getFirst() {
-    return first;
-  }
-
-  public void setFirst(F first) {
-    this.first = first;
-  }
-
-  public S getSecond() {
-    return second;
-  }
-
-  public void setSecond(S second) {
-    this.second = second;
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null) {
-      return false;
-    }
-    if (that instanceof ObjectPair) {
-      return this.equals((ObjectPair<F, S>)that);
-    }
-    return false;
-  }
-
-  public boolean equals(ObjectPair<F, S> that) {
-    if (that == null) {
-      return false;
-    }
-
-    return this.getFirst().equals(that.getFirst()) &&
-        this.getSecond().equals(that.getSecond());
-  }
-
-  @Override
-  public int hashCode() {
-    return first.hashCode() * 31 + second.hashCode();
-  }
-
-  public String toString() {
-    return first + ":" + second;
-  }
-}
diff --git 
a/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
 
b/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
index a06191d..1cb35bb 100644
--- 
a/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
+++ 
b/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
@@ -27,6 +27,7 @@ import java.util.Map;
 
 import javax.annotation.Nullable;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
@@ -51,7 +52,6 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -584,7 +584,7 @@ public class HCatClientHMSImpl extends HCatClient {
       throws SemanticException, TException {
     LOG.info("HCatClient: Dropping partitions using partition-predicate 
Expressions.");
     ExprNodeGenericFuncDesc partitionExpression = new ExpressionBuilder(table, 
partitionSpec).build();
-    ObjectPair<Integer, byte[]> serializedPartitionExpression = new 
ObjectPair<>(partitionSpec.size(),
+    Pair<Integer, byte[]> serializedPartitionExpression = 
Pair.of(partitionSpec.size(),
             
SerializationUtilities.serializeExpressionToKryo(partitionExpression));
     hmsClient.dropPartitions(table.getDbName(), table.getTableName(), 
Arrays.asList(serializedPartitionExpression),
         deleteData && !isExternal(table),  // Delete data?
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
index 13c9416..6f0dfba 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
@@ -21,9 +21,9 @@ package org.apache.hadoop.hive.ql.ddl.table.partition;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.metastore.PartitionDropOptions;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.DDLUtils;
@@ -106,9 +106,9 @@ public class AlterTableDropPartitionOperation extends 
DDLOperation<AlterTableDro
     // ifExists is currently verified in DDLSemanticAnalyzer
     String[] names = Utilities.getDbTableName(desc.getTableName());
 
-    List<ObjectPair<Integer, byte[]>> partitionExpressions = new 
ArrayList<>(desc.getPartSpecs().size());
+    List<Pair<Integer, byte[]>> partitionExpressions = new 
ArrayList<>(desc.getPartSpecs().size());
     for (AlterTableDropPartitionDesc.PartitionDesc partSpec : 
desc.getPartSpecs()) {
-      partitionExpressions.add(new ObjectPair<>(partSpec.getPrefixLength(),
+      partitionExpressions.add(Pair.of(partSpec.getPrefixLength(),
           
SerializationUtilities.serializeExpressionToKryo(partSpec.getPartSpec())));
     }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FooterBuffer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/FooterBuffer.java
index 8ead797..8668b1d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FooterBuffer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FooterBuffer.java
@@ -20,8 +20,9 @@ package org.apache.hadoop.hive.ql.exec;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.List;
 
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobConf;
@@ -29,7 +30,7 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.util.ReflectionUtils;
 
 public class FooterBuffer {
-  private ArrayList<ObjectPair<WritableComparable, Writable>> buffer;
+  private List<Pair<WritableComparable, Writable>> buffer;
   private int cur;
 
   public FooterBuffer() {
@@ -70,9 +71,10 @@ public class FooterBuffer {
       if (!notEOF) {
         return false;
       }
-      ObjectPair<WritableComparable, Writable> tem = new ObjectPair<>();
-      tem.setFirst(ReflectionUtils.copy(job, key, tem.getFirst()));
-      tem.setSecond(ReflectionUtils.copy(job, value, tem.getSecond()));
+
+      WritableComparable left = ReflectionUtils.copy(job, key, null);
+      Writable right = ReflectionUtils.copy(job, value, null);
+      Pair<WritableComparable, Writable> tem = Pair.of(left, right);
       buffer.add(tem);
     }
     this.cur = 0;
@@ -98,9 +100,9 @@ public class FooterBuffer {
    */
   public boolean updateBuffer(JobConf job, RecordReader recordreader,
       WritableComparable key, Writable value) throws IOException {
-    key = ReflectionUtils.copy(job, buffer.get(cur).getFirst(), key);
-    value = ReflectionUtils.copy(job, buffer.get(cur).getSecond(), value);
-    boolean notEOF = recordreader.next(buffer.get(cur).getFirst(), 
buffer.get(cur).getSecond());
+    key = ReflectionUtils.copy(job, buffer.get(cur).getKey(), key);
+    value = ReflectionUtils.copy(job, buffer.get(cur).getValue(), value);
+    boolean notEOF = recordreader.next(buffer.get(cur).getKey(), 
buffer.get(cur).getValue());
     if (notEOF) {
       cur = (++cur) % buffer.size();
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index b0b4ec8..0643a54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -23,13 +23,11 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Future;
-import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -134,7 +132,6 @@ public class MapJoinOperator extends 
AbstractMapJoinOperator<MapJoinDesc> implem
   // Only used in bucket map join.
   private transient int numBuckets = -1;
   private transient int bucketId = -1;
-  private transient ReentrantLock subCacheLock = new ReentrantLock();
 
   /** Kryo ctor. */
   protected MapJoinOperator() {
@@ -680,8 +677,6 @@ public class MapJoinOperator extends 
AbstractMapJoinOperator<MapJoinDesc> implem
      */
     NonMatchedSmallTableIterator nonMatchedIterator =
         substituteSmallTable.createNonMatchedSmallTableIterator(matchTracker);
-    int nonMatchedKeyCount = 0;
-    int nonMatchedValueCount = 0;
     while (nonMatchedIterator.isNext()) {
       List<Object> keyObjList = nonMatchedIterator.getCurrentKey();
 
@@ -729,10 +724,7 @@ public class MapJoinOperator extends 
AbstractMapJoinOperator<MapJoinDesc> implem
 
         // FUTURE: Support residual filters for non-equi joins.
         internalForward(standardCopyRow, outputObjInspector);
-        nonMatchedValueCount++;
       }
-
-      nonMatchedKeyCount++;
     }
   }
 
@@ -938,9 +930,9 @@ public class MapJoinOperator extends 
AbstractMapJoinOperator<MapJoinDesc> implem
 
     KeyValueHelper writeHelper = container.getWriteHelper();
     while (kvContainer.hasNext()) {
-      ObjectPair<HiveKey, BytesWritable> pair = kvContainer.next();
-      Writable key = pair.getFirst();
-      Writable val = pair.getSecond();
+      Pair<HiveKey, BytesWritable> pair = kvContainer.next();
+      Writable key = pair.getLeft();
+      Writable val = pair.getRight();
       writeHelper.setKeyValue(key, val);
       restoredHashMap.put(writeHelper, -1);
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
index bfdb7d2..c09bf53 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
@@ -28,9 +28,9 @@ import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.lang3.tuple.MutablePair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
@@ -696,7 +696,7 @@ public class SMBMapJoinOperator extends 
AbstractMapJoinOperator<SMBJoinDesc> imp
 
     // index of FetchOperator which is providing smallest one
     transient Integer currentMinSegment;
-    transient ObjectPair<List<Object>, InspectableObject>[] keys;
+    transient MutablePair<List<Object>, InspectableObject>[] keys;
 
     public MergeQueue(String alias, FetchWork fetchWork, JobConf jobConf,
         Operator<? extends OperatorDesc> forwardOp,
@@ -737,7 +737,7 @@ public class SMBMapJoinOperator extends 
AbstractMapJoinOperator<SMBJoinDesc> imp
     private FetchOperator[] segmentsForSize(int segmentLen) {
       if (segments == null || segments.length < segmentLen) {
         FetchOperator[] newSegments = new FetchOperator[segmentLen];
-        ObjectPair<List<Object>, InspectableObject>[] newKeys = new 
ObjectPair[segmentLen];
+        MutablePair<List<Object>, InspectableObject>[] newKeys = new 
MutablePair[segmentLen];
         if (segments != null) {
           System.arraycopy(segments, 0, newSegments, 0, segments.length);
           System.arraycopy(keys, 0, newKeys, 0, keys.length);
@@ -760,7 +760,7 @@ public class SMBMapJoinOperator extends 
AbstractMapJoinOperator<SMBJoinDesc> imp
 
     @Override
     protected boolean lessThan(Object a, Object b) {
-      return compareKeys(keys[(Integer) a].getFirst(), 
keys[(Integer)b].getFirst()) < 0;
+      return compareKeys(keys[(Integer) a].getLeft(), 
keys[(Integer)b].getLeft()) < 0;
     }
 
     public final InspectableObject getNextRow() throws IOException {
@@ -775,7 +775,8 @@ public class SMBMapJoinOperator extends 
AbstractMapJoinOperator<SMBJoinDesc> imp
         return null;
       }
       counter++;
-      return keys[currentMinSegment = current].getSecond();
+      currentMinSegment = current;
+      return keys[currentMinSegment].getRight();
     }
 
     private void adjustPriorityQueue(Integer current) throws IOException {
@@ -816,7 +817,7 @@ public class SMBMapJoinOperator extends 
AbstractMapJoinOperator<SMBJoinDesc> imp
       while (nextRow != null) {
         sinkOp.reset();
         if (keys[current] == null) {
-          keys[current] = new ObjectPair<List<Object>, InspectableObject>();
+          keys[current] = new MutablePair<List<Object>, InspectableObject>();
         }
 
         // Pass the row though the operator tree. It is guaranteed that not 
more than 1 row can
@@ -827,8 +828,8 @@ public class SMBMapJoinOperator extends 
AbstractMapJoinOperator<SMBJoinDesc> imp
         // It is possible that the row got absorbed in the operator tree.
         if (nextRow.o != null) {
           // todo this should be changed to be evaluated lazily, especially 
for single segment case
-          keys[current].setFirst(JoinUtil.computeKeys(nextRow.o, keyFields, 
keyFieldOIs));
-          keys[current].setSecond(nextRow);
+          keys[current].setLeft(JoinUtil.computeKeys(nextRow.o, keyFields, 
keyFieldOIs));
+          keys[current].setRight(nextRow);
           return true;
         }
         nextRow = segments[current].getNextRow();
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
index 3f00dc6..6d06bff 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/KeyValueContainer.java
@@ -23,9 +23,9 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.lang3.tuple.MutablePair;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.io.BytesWritable;
@@ -47,7 +47,7 @@ public class KeyValueContainer {
   @VisibleForTesting
   static final int IN_MEMORY_NUM_ROWS = 1024;
 
-  private ObjectPair<HiveKey, BytesWritable>[] readBuffer;
+  private MutablePair<HiveKey, BytesWritable>[] readBuffer;
   private boolean readBufferUsed = false; // indicates if read buffer has data
   private int rowsInReadBuffer = 0;       // number of rows in the temporary 
read buffer
   private int readCursor = 0;             // cursor during reading
@@ -60,9 +60,9 @@ public class KeyValueContainer {
   private Output output;
 
   public KeyValueContainer(String spillLocalDirs) {
-    readBuffer = new ObjectPair[IN_MEMORY_NUM_ROWS];
+    readBuffer = new MutablePair[IN_MEMORY_NUM_ROWS];
     for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) {
-      readBuffer[i] = new ObjectPair<HiveKey, BytesWritable>();
+      readBuffer[i] = new MutablePair<HiveKey, BytesWritable>();
     }
     try {
       setupOutput(spillLocalDirs);
@@ -158,7 +158,7 @@ public class KeyValueContainer {
     return readBufferUsed || rowsOnDisk > 0;
   }
 
-  public ObjectPair<HiveKey, BytesWritable> next() {
+  public MutablePair<HiveKey, BytesWritable> next() {
     Preconditions.checkState(hasNext());
     if (!readBufferUsed) {
       try {
@@ -186,9 +186,9 @@ public class KeyValueContainer {
           }
 
           for (int i = 0; i < rowsInReadBuffer; i++) {
-            ObjectPair<HiveKey, BytesWritable> pair = readBuffer[i];
-            pair.setFirst(readHiveKey(input));
-            pair.setSecond(readValue(input));
+            MutablePair<HiveKey, BytesWritable> pair = readBuffer[i];
+            pair.setLeft(readHiveKey(input));
+            pair.setRight(readValue(input));
           }
 
           if (input.eof()) {
@@ -206,7 +206,7 @@ public class KeyValueContainer {
       }
     }
 
-    ObjectPair<HiveKey, BytesWritable> row = readBuffer[readCursor];
+    MutablePair<HiveKey, BytesWritable> row = readBuffer[readCursor];
     if (++readCursor >= rowsInReadBuffer) {
       readBufferUsed = false;
       rowsInReadBuffer = 0;
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java
index 413946b..253e310 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java
@@ -24,8 +24,8 @@ import java.io.IOException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.lang3.tuple.MutablePair;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.io.BytesWritable;
 
@@ -49,8 +49,8 @@ class HiveKVResultCache {
   @VisibleForTesting
   static final int IN_MEMORY_NUM_ROWS = 1024;
 
-  private ObjectPair<HiveKey, BytesWritable>[] writeBuffer;
-  private ObjectPair<HiveKey, BytesWritable>[] readBuffer;
+  private MutablePair<HiveKey, BytesWritable>[] writeBuffer;
+  private MutablePair<HiveKey, BytesWritable>[] readBuffer;
 
   private File parentFile;
   private File tmpFile;
@@ -67,16 +67,16 @@ class HiveKVResultCache {
   private Output output;
 
   public HiveKVResultCache() {
-    writeBuffer = new ObjectPair[IN_MEMORY_NUM_ROWS];
-    readBuffer = new ObjectPair[IN_MEMORY_NUM_ROWS];
+    writeBuffer = new MutablePair[IN_MEMORY_NUM_ROWS];
+    readBuffer = new MutablePair[IN_MEMORY_NUM_ROWS];
     for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) {
-      writeBuffer[i] = new ObjectPair<HiveKey, BytesWritable>();
-      readBuffer[i] = new ObjectPair<HiveKey, BytesWritable>();
+      writeBuffer[i] = new MutablePair<HiveKey, BytesWritable>();
+      readBuffer[i] = new MutablePair<HiveKey, BytesWritable>();
     }
   }
 
   private void switchBufferAndResetCursor() {
-    ObjectPair<HiveKey, BytesWritable>[] tmp = readBuffer;
+    MutablePair<HiveKey, BytesWritable>[] tmp = readBuffer;
     rowsInReadBuffer = writeCursor;
     readBuffer = writeBuffer;
     readBufferUsed = true;
@@ -152,11 +152,11 @@ class HiveKVResultCache {
             setupOutput();
           }
           for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) {
-            ObjectPair<HiveKey, BytesWritable> pair = writeBuffer[i];
-            writeHiveKey(output, pair.getFirst());
-            writeValue(output, pair.getSecond());
-            pair.setFirst(null);
-            pair.setSecond(null);
+            MutablePair<HiveKey, BytesWritable> pair = writeBuffer[i];
+            writeHiveKey(output, pair.getLeft());
+            writeValue(output, pair.getRight());
+            pair.setLeft(null);
+            pair.setRight(null);
           }
           writeCursor = 0;
         } catch (Exception e) {
@@ -165,9 +165,9 @@ class HiveKVResultCache {
         }
       }
     }
-    ObjectPair<HiveKey, BytesWritable> pair = writeBuffer[writeCursor++];
-    pair.setFirst(key);
-    pair.setSecond(value);
+    MutablePair<HiveKey, BytesWritable> pair = writeBuffer[writeCursor++];
+    pair.setLeft(key);
+    pair.setRight(value);
   }
 
   public synchronized void clear() {
@@ -224,9 +224,9 @@ class HiveKVResultCache {
         if (input != null) {
           // Load next batch from disk
           for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) {
-            ObjectPair<HiveKey, BytesWritable> pair = readBuffer[i];
-            pair.setFirst(readHiveKey(input));
-            pair.setSecond(readValue(input));
+            MutablePair<HiveKey, BytesWritable> pair = readBuffer[i];
+            pair.setLeft(readHiveKey(input));
+            pair.setRight(readValue(input));
           }
           if (input.eof()) {
             input.close();
@@ -236,11 +236,10 @@ class HiveKVResultCache {
           readBufferUsed = true;
           readCursor = 0;
         } else if (writeCursor == 1) {
-          ObjectPair<HiveKey, BytesWritable> pair = writeBuffer[0];
-          Tuple2<HiveKey, BytesWritable> row = new Tuple2<HiveKey, 
BytesWritable>(
-            pair.getFirst(), pair.getSecond());
-          pair.setFirst(null);
-          pair.setSecond(null);
+          MutablePair<HiveKey, BytesWritable> pair = writeBuffer[0];
+          Tuple2<HiveKey, BytesWritable> row = new Tuple2<HiveKey, 
BytesWritable>(pair.getLeft(), pair.getRight());
+          pair.setLeft(null);
+          pair.setRight(null);
           writeCursor = 0;
           return row;
         } else {
@@ -252,11 +251,10 @@ class HiveKVResultCache {
         throw new RuntimeException("Failed to load rows from disk", e);
       }
     }
-    ObjectPair<HiveKey, BytesWritable> pair = readBuffer[readCursor];
-    Tuple2<HiveKey, BytesWritable> row = new Tuple2<HiveKey, BytesWritable>(
-      pair.getFirst(), pair.getSecond());
-    pair.setFirst(null);
-    pair.setSecond(null);
+    MutablePair<HiveKey, BytesWritable> pair = readBuffer[readCursor];
+    Tuple2<HiveKey, BytesWritable> row = new Tuple2<HiveKey, 
BytesWritable>(pair.getLeft(), pair.getRight());
+    pair.setLeft(null);
+    pair.setRight(null);
     if (++readCursor >= rowsInReadBuffer) {
       readBufferUsed = false;
       rowsInReadBuffer = 0;
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java
index 62f88c4..b29be48 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hive.ql.exec.spark.session;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef;
@@ -50,7 +50,7 @@ public interface SparkSession {
    * @return an object pair, the first element is the shuffle memory per task 
in bytes,
    *  the second element is the number of total cores usable by the client
    */
-  ObjectPair<Long, Integer> getMemoryAndCores() throws Exception;
+  Pair<Long, Integer> getMemoryAndCores() throws Exception;
 
   /**
    * @return true if the session is open and ready to submit jobs.
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
index 1d251ed..7e64342 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
@@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -41,7 +42,6 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClient;
@@ -153,14 +153,14 @@ public class SparkSessionImpl implements SparkSession {
   }
 
   @Override
-  public ObjectPair<Long, Integer> getMemoryAndCores() throws Exception {
+  public Pair<Long, Integer> getMemoryAndCores() throws Exception {
     closeLock.readLock().lock();
     try {
       SparkConf sparkConf = hiveSparkClient.getSparkConf();
       int numExecutors = hiveSparkClient.getExecutorCount();
       // at start-up, we may be unable to get number of executors
       if (numExecutors <= 0) {
-        return new ObjectPair<Long, Integer>(-1L, -1);
+        return Pair.of(-1L, -1);
       }
       int executorMemoryInMB = Utils.memoryStringToMb(
               sparkConf.get("spark.executor.memory", "512m"));
@@ -183,8 +183,7 @@ public class SparkSessionImpl implements SparkSession {
       LOG.info("Hive on Spark application currently has number of executors: " 
+ numExecutors
               + ", total cores: " + totalCores + ", memory per executor: "
               + executorMemoryInMB + " mb, memoryFraction: " + memoryFraction);
-      return new ObjectPair<Long, Integer>(Long.valueOf(memoryPerTaskInBytes),
-              Integer.valueOf(totalCores));
+      return Pair.of(Long.valueOf(memoryPerTaskInBytes), 
Integer.valueOf(totalCores));
     } finally {
       closeLock.readLock().unlock();
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java 
b/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
index a9d845a..ec4c04c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
@@ -26,7 +26,7 @@ import com.google.gson.stream.JsonWriter;
 import org.apache.commons.collections.SetUtils;
 import org.apache.commons.io.output.StringBuilderWriter;
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -249,15 +249,15 @@ public class LineageLogger implements 
ExecuteWithHookContext {
    */
   @VisibleForTesting
   public static List<Edge> getEdges(QueryPlan plan, Index index) {
-    LinkedHashMap<String, ObjectPair<SelectOperator,
-      org.apache.hadoop.hive.ql.metadata.Table>> finalSelOps = 
index.getFinalSelectOps();
+    Map<String, Pair<SelectOperator, 
org.apache.hadoop.hive.ql.metadata.Table>> finalSelOps =
+        index.getFinalSelectOps();
     Map<String, Vertex> vertexCache = new LinkedHashMap<String, Vertex>();
     List<Edge> edges = new ArrayList<Edge>();
-    for (ObjectPair<SelectOperator,
+    for (Pair<SelectOperator,
         org.apache.hadoop.hive.ql.metadata.Table> pair: finalSelOps.values()) {
       List<FieldSchema> fieldSchemas = 
plan.getResultSchema().getFieldSchemas();
-      SelectOperator finalSelOp = pair.getFirst();
-      org.apache.hadoop.hive.ql.metadata.Table t = pair.getSecond();
+      SelectOperator finalSelOp = pair.getLeft();
+      org.apache.hadoop.hive.ql.metadata.Table t = pair.getRight();
       String destTableName = null;
       List<String> colNames = null;
       if (t != null) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java 
b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 522c20a..db8cc6c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -80,6 +80,7 @@ import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.lang3.exception.ExceptionUtils;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
@@ -90,7 +91,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.HiveStatsUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
@@ -115,8 +115,64 @@ import 
org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
 import org.apache.hadoop.hive.metastore.SynchronizedMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventRequestData;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.InsertEventRequestData;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.Materialization;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.ReplChangeManager;
-import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
@@ -3322,7 +3378,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   public List<Partition> dropPartitions(String dbName, String tableName,
-      List<org.apache.hadoop.hive.metastore.utils.ObjectPair<Integer, byte[]>> 
partitionExpressions,
+      List<Pair<Integer, byte[]>> partitionExpressions,
       PartitionDropOptions dropOptions) throws HiveException {
     try {
       Table table = getTable(dbName, tableName);
@@ -3810,7 +3866,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
     if (!fullDestStatus.getFileStatus().isDirectory()) {
       throw new HiveException(destf + " is not a directory.");
     }
-    final List<Future<ObjectPair<Path, Path>>> futures = new LinkedList<>();
+    final List<Future<Pair<Path, Path>>> futures = new LinkedList<>();
     final ExecutorService pool = 
conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 25) > 0 ?
         
Executors.newFixedThreadPool(conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname,
 25),
         new 
ThreadFactoryBuilder().setDaemon(true).setNameFormat("Move-Thread-%d").build()) 
: null;
@@ -3864,9 +3920,9 @@ private void constructOneLBLocationMap(FileStatus fSta,
         } else {
           // future only takes final or seemingly final values. Make a final 
copy of taskId
           final int finalTaskId = acidRename ? taskId++ : -1;
-          futures.add(pool.submit(new Callable<ObjectPair<Path, Path>>() {
+          futures.add(pool.submit(new Callable<Pair<Path, Path>>() {
             @Override
-            public ObjectPair<Path, Path> call() throws HiveException {
+            public Pair<Path, Path> call() throws HiveException {
               SessionState.setCurrentSessionState(parentSession);
 
               try {
@@ -3876,7 +3932,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
                 if (null != newFiles) {
                   newFiles.add(destPath);
                 }
-                return ObjectPair.create(srcP, destPath);
+                return Pair.of(srcP, destPath);
               } catch (Exception e) {
                 throw getHiveException(e, msg);
               }
@@ -3887,10 +3943,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
     }
     if (null != pool) {
       pool.shutdown();
-      for (Future<ObjectPair<Path, Path>> future : futures) {
+      for (Future<Pair<Path, Path>> future : futures) {
         try {
-          ObjectPair<Path, Path> pair = future.get();
-          LOG.debug("Moved src: {}, to dest: {}", pair.getFirst().toString(), 
pair.getSecond().toString());
+          Pair<Path, Path> pair = future.get();
+          LOG.debug("Moved src: {}, to dest: {}", pair.getLeft().toString(), 
pair.getRight().toString());
         } catch (Exception e) {
           throw handlePoolException(pool, e);
         }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index de61be8..c937b62 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -522,7 +522,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       mapJoinDesc =
           new MapJoinDesc(
                   MapJoinProcessor.getKeys(joinOp.getConf().isLeftInputJoin(),
-                  joinOp.getConf().getBaseSrc(), joinOp).getSecond(),
+                  joinOp.getConf().getBaseSrc(), joinOp).getRight(),
                   null, joinDesc.getExprs(), null, null,
                   joinDesc.getOutputColumnNames(), mapJoinConversionPos, 
joinDesc.getConds(),
                   joinDesc.getFilters(), joinDesc.getNoOuterJoin(), null,
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
index 5ed43c7..a9506c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
@@ -30,8 +30,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
@@ -561,8 +561,6 @@ public class MapJoinProcessor extends Transform {
 
   public static boolean isFullOuterEnabledForDynamicPartitionHashJoin(HiveConf 
hiveConf, JoinOperator joinOp)
       throws SemanticException {
-    JoinDesc joinDesc = joinOp.getConf();
-
     return true;
   }
 
@@ -1230,7 +1228,7 @@ public class MapJoinProcessor extends Transform {
 
   }
 
-  public static ObjectPair<List<ReduceSinkOperator>, Map<Byte, 
List<ExprNodeDesc>>> getKeys(
+  public static Pair<List<ReduceSinkOperator>, Map<Byte, List<ExprNodeDesc>>> 
getKeys(
           boolean leftInputJoin, String[] baseSrc, JoinOperator op) {
 
     // Walk over all the sources (which are guaranteed to be reduce sink
@@ -1264,8 +1262,7 @@ public class MapJoinProcessor extends Transform {
       keyExprMap.put(pos, keyCols);
     }
 
-    return new ObjectPair<List<ReduceSinkOperator>, 
Map<Byte,List<ExprNodeDesc>>>(
-            oldReduceSinkParentOps, keyExprMap);
+    return Pair.of(oldReduceSinkParentOps, keyExprMap);
   }
 
   public static MapJoinDesc getMapJoinDesc(HiveConf hconf,
@@ -1287,9 +1284,8 @@ public class MapJoinProcessor extends Transform {
     Map<Byte, List<ExprNodeDesc>> valueExprs = op.getConf().getExprs();
     Map<Byte, List<ExprNodeDesc>> newValueExprs = new HashMap<Byte, 
List<ExprNodeDesc>>();
 
-    ObjectPair<List<ReduceSinkOperator>, Map<Byte,List<ExprNodeDesc>>> pair =
-            getKeys(leftInputJoin, baseSrc, op);
-    List<ReduceSinkOperator> oldReduceSinkParentOps = pair.getFirst();
+    Pair<List<ReduceSinkOperator>, Map<Byte, List<ExprNodeDesc>>> pair = 
getKeys(leftInputJoin, baseSrc, op);
+    List<ReduceSinkOperator> oldReduceSinkParentOps = pair.getLeft();
     for (Map.Entry<Byte, List<ExprNodeDesc>> entry : valueExprs.entrySet()) {
       byte tag = entry.getKey();
       Operator<?> terminal = oldReduceSinkParentOps.get(tag);
@@ -1318,7 +1314,7 @@ public class MapJoinProcessor extends Transform {
     Map<Byte, int[]> valueIndices = new HashMap<Byte, int[]>();
 
     // get the join keys from old parent ReduceSink operators
-    Map<Byte, List<ExprNodeDesc>> keyExprMap = pair.getSecond();
+    Map<Byte, List<ExprNodeDesc>> keyExprMap = pair.getRight();
 
     if (!adjustParentsChildren) {
       // Since we did not remove reduce sink parents, keep the original value 
expressions
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/LineageCtx.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/LineageCtx.java
index ad1dec9..58b90d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/LineageCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/lineage/LineageCtx.java
@@ -26,7 +26,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -55,8 +55,7 @@ public class LineageCtx implements NodeProcessorCtx {
      * dependency vector for that tuple. This is used to generate the
      * dependency vectors during the walk of the operator tree.
      */
-    private final Map<Operator<? extends OperatorDesc>,
-                      LinkedHashMap<ColumnInfo, Dependency>> depMap;
+    private final Map<Operator<? extends OperatorDesc>, Map<ColumnInfo, 
Dependency>> depMap;
 
     /**
      * A map from operator to the conditions strings.
@@ -67,18 +66,17 @@ public class LineageCtx implements NodeProcessorCtx {
      * A map from a final select operator id to the select operator
      * and the corresponding target table in case an insert into query.
      */
-    private LinkedHashMap<String, ObjectPair<SelectOperator, Table>> 
finalSelectOps;
+    private Map<String, Pair<SelectOperator, Table>> finalSelectOps;
 
     /**
      * Constructor.
      */
     public Index() {
       depMap =
-        new LinkedHashMap<Operator<? extends OperatorDesc>,
-                          LinkedHashMap<ColumnInfo, Dependency>>();
+        new LinkedHashMap<Operator<? extends OperatorDesc>, Map<ColumnInfo, 
Dependency>>();
       condMap = new HashMap<Operator<? extends OperatorDesc>, 
Set<Predicate>>();
       finalSelectOps =
-        new LinkedHashMap<String, ObjectPair<SelectOperator, Table>>();
+        new LinkedHashMap<String, Pair<SelectOperator, Table>>();
     }
 
     /**
@@ -128,7 +126,7 @@ public class LineageCtx implements NodeProcessorCtx {
      */
     public void putDependency(Operator<? extends OperatorDesc> op,
         ColumnInfo col, Dependency dep) {
-      LinkedHashMap<ColumnInfo, Dependency> colMap = depMap.get(op);
+      Map<ColumnInfo, Dependency> colMap = depMap.get(op);
       if (colMap == null) {
         colMap = new LinkedHashMap<ColumnInfo, Dependency>();
         depMap.put(op, colMap);
@@ -204,13 +202,11 @@ public class LineageCtx implements NodeProcessorCtx {
           FileSinkOperator fso = (FileSinkOperator) sinkOp;
           table = fso.getConf().getTable();
         }
-        finalSelectOps.put(operatorId,
-          new ObjectPair<SelectOperator, Table>(sop, table));
+        finalSelectOps.put(operatorId, Pair.of(sop, table));
       }
     }
 
-    public LinkedHashMap<String,
-        ObjectPair<SelectOperator, Table>> getFinalSelectOps() {
+    public Map<String, Pair<SelectOperator, Table>> getFinalSelectOps() {
       return finalSelectOps;
     }
 
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
index 636e0a2..1b9db01 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartExprEvalUtils.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -102,7 +102,7 @@ public class PartExprEvalUtils {
         .getPrimitiveJavaObject(evaluateResultO);
   }
 
-  static public ObjectPair<PrimitiveObjectInspector, ExprNodeEvaluator> 
prepareExpr(
+  public static Pair<PrimitiveObjectInspector, ExprNodeEvaluator> prepareExpr(
       ExprNodeGenericFuncDesc expr, List<String> partColumnNames,
       List<PrimitiveTypeInfo> partColumnTypeInfos) throws HiveException {
     // Create the row object
@@ -116,12 +116,12 @@ public class PartExprEvalUtils {
 
     ExprNodeEvaluator evaluator = ExprNodeEvaluatorFactory.get(expr);
     ObjectInspector evaluateResultOI = evaluator.initialize(objectInspector);
-    return ObjectPair.create((PrimitiveObjectInspector)evaluateResultOI, 
evaluator);
+    return Pair.of((PrimitiveObjectInspector)evaluateResultOI, evaluator);
   }
 
   static public Object evaluateExprOnPart(
-      ObjectPair<PrimitiveObjectInspector, ExprNodeEvaluator> pair, Object 
partColValues)
+      Pair<PrimitiveObjectInspector, ExprNodeEvaluator> pair, Object 
partColValues)
           throws HiveException {
-    return 
pair.getFirst().getPrimitiveJavaObject(pair.getSecond().evaluate(partColValues));
+    return 
pair.getLeft().getPrimitiveJavaObject(pair.getRight().evaluate(partColValues));
   }
 }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
index 7c8d752..673d858 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
@@ -27,7 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
@@ -551,7 +551,7 @@ public class PartitionPruner extends Transform {
       List<PrimitiveTypeInfo> partColumnTypeInfos, ExprNodeGenericFuncDesc 
prunerExpr,
       String defaultPartitionName, List<String> partNames) throws 
HiveException, MetaException {
     // Prepare the expression to filter on the columns.
-    ObjectPair<PrimitiveObjectInspector, ExprNodeEvaluator> handle =
+    Pair<PrimitiveObjectInspector, ExprNodeEvaluator> handle =
         PartExprEvalUtils.prepareExpr(prunerExpr, partColumnNames, 
partColumnTypeInfos);
 
     // Filter the name list. Removing elements one by one can be slow on e.g. 
ArrayList,
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
index ab87c79..341422f 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java
@@ -27,7 +27,7 @@ import java.util.Stack;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
@@ -66,7 +66,7 @@ public class SetSparkReducerParallelism implements 
NodeProcessor {
   private static final String SPARK_DYNAMIC_ALLOCATION_ENABLED = 
"spark.dynamicAllocation.enabled";
 
   // Spark memory per task, and total number of cores
-  private ObjectPair<Long, Integer> sparkMemoryAndCores;
+  private Pair<Long, Integer> sparkMemoryAndCores;
   private final boolean useOpStats;
 
   public SetSparkReducerParallelism(HiveConf conf) {
@@ -169,15 +169,15 @@ public class SetSparkReducerParallelism implements 
NodeProcessor {
 
           getSparkMemoryAndCores(context);
           if (sparkMemoryAndCores != null &&
-              sparkMemoryAndCores.getFirst() > 0 && 
sparkMemoryAndCores.getSecond() > 0) {
+              sparkMemoryAndCores.getLeft() > 0 && 
sparkMemoryAndCores.getRight() > 0) {
             // warn the user if bytes per reducer is much larger than memory 
per task
-            if ((double) sparkMemoryAndCores.getFirst() / bytesPerReducer < 
0.5) {
+            if ((double) sparkMemoryAndCores.getLeft() / bytesPerReducer < 
0.5) {
               LOG.warn("Average load of a reducer is much larger than its 
available memory. " +
                   "Consider decreasing hive.exec.reducers.bytes.per.reducer");
             }
 
             // If there are more cores, use the number of cores
-            numReducers = Math.max(numReducers, 
sparkMemoryAndCores.getSecond());
+            numReducers = Math.max(numReducers, 
sparkMemoryAndCores.getRight());
           }
           numReducers = Math.min(numReducers, maxReducers);
           LOG.info("Set parallelism for reduce sink " + sink + " to: " + 
numReducers +
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 8d9718f..43dfcee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -3825,7 +3825,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         int groupingColsSize = gbExprNDescLst.size();
         List<Long> groupingSets = null;
         if (cubeRollupGrpSetPresent) {
-          groupingSets = getGroupByGroupingSetsForClause(qbp, 
detsClauseName).getSecond();
+          groupingSets = getGroupByGroupingSetsForClause(qbp, 
detsClauseName).getRight();
         }
 
         // 6. Construct aggregation function Info
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
index 51448a4..200e814 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/QBSubQuery.java
@@ -23,7 +23,7 @@ import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
@@ -279,14 +279,14 @@ public class QBSubQuery implements ISubQueryJoinInfo {
      * 3. All other expressions have a Type based on their children.
      *    An Expr w/o children is assumed to refer to neither.
      */
-    private ObjectPair<ExprType,ColumnInfo> analyzeExpr(ASTNode expr) {
+    private Pair<ExprType, ColumnInfo> analyzeExpr(ASTNode expr) {
       ColumnInfo cInfo = null;
       if ( forHavingClause ) {
         try {
           cInfo = parentQueryRR.getExpression(expr);
           if ( cInfo != null) {
-              return ObjectPair.create(ExprType.REFERS_PARENT, cInfo);
-            }
+            return Pair.of(ExprType.REFERS_PARENT, cInfo);
+          }
         } catch(SemanticException se) {
         }
       }
@@ -294,19 +294,19 @@ public class QBSubQuery implements ISubQueryJoinInfo {
         ASTNode dot = firstDot(expr);
         cInfo = resolveDot(dot);
         if ( cInfo != null ) {
-          return ObjectPair.create(ExprType.REFERS_PARENT, cInfo);
+          return Pair.of(ExprType.REFERS_PARENT, cInfo);
         }
-        return ObjectPair.create(ExprType.REFERS_SUBQUERY, null);
+        return Pair.of(ExprType.REFERS_SUBQUERY, null);
       } else if ( expr.getType() == HiveParser.TOK_TABLE_OR_COL ) {
-        return ObjectPair.create(ExprType.REFERS_SUBQUERY, null);
+        return Pair.of(ExprType.REFERS_SUBQUERY, null);
       } else {
         ExprType exprType = ExprType.REFERS_NONE;
         int cnt = expr.getChildCount();
         for(int i=0; i < cnt; i++) {
           ASTNode child = (ASTNode) expr.getChild(i);
-          exprType = exprType.combine(analyzeExpr(child).getFirst());
+          exprType = exprType.combine(analyzeExpr(child).getLeft());
         }
-        return ObjectPair.create(exprType, null);
+        return Pair.of(exprType, null);
       }
     }
 
@@ -321,17 +321,17 @@ public class QBSubQuery implements ISubQueryJoinInfo {
        if(conjunct.getChildCount() == 2) {
         ASTNode left = (ASTNode) conjunct.getChild(0);
         ASTNode right = (ASTNode) conjunct.getChild(1);
-        ObjectPair<ExprType,ColumnInfo> leftInfo = analyzeExpr(left);
-        ObjectPair<ExprType,ColumnInfo> rightInfo = analyzeExpr(right);
+        Pair<ExprType, ColumnInfo> leftInfo = analyzeExpr(left);
+        Pair<ExprType, ColumnInfo> rightInfo = analyzeExpr(right);
 
         return new Conjunct(left, right,
-            leftInfo.getFirst(), rightInfo.getFirst(),
-            leftInfo.getSecond(), rightInfo.getSecond());
+            leftInfo.getLeft(), rightInfo.getLeft(),
+            leftInfo.getRight(), rightInfo.getRight());
       } else {
-        ObjectPair<ExprType,ColumnInfo> sqExprInfo = analyzeExpr(conjunct);
+        Pair<ExprType, ColumnInfo> sqExprInfo = analyzeExpr(conjunct);
         return new Conjunct(conjunct, null,
-            sqExprInfo.getFirst(), null,
-            sqExprInfo.getSecond(), sqExprInfo.getSecond());
+            sqExprInfo.getLeft(), null,
+            sqExprInfo.getRight(), sqExprInfo.getRight());
       }
     }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 4548276..050d09d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -67,7 +67,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.StatsSetupConst.StatDB;
 import org.apache.hadoop.hive.common.StringInternUtils;
@@ -2094,8 +2093,8 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
     // For eg: for a query like 'select * from V3', where V3 -> V2, V2 -> V1, 
V1 -> T
     // keeps track of full view name and read entity corresponding to alias 
V3, V3:V2, V3:V2:V1.
     // This is needed for tracking the dependencies for inputs, along with 
their parents.
-    Map<String, ObjectPair<String, ReadEntity>> aliasToViewInfo =
-        new HashMap<String, ObjectPair<String, ReadEntity>>();
+    Map<String, Pair<String, ReadEntity>> aliasToViewInfo =
+        new HashMap<String, Pair<String, ReadEntity>>();
 
     /*
      * used to capture view to SQ conversions. This is used to check for
@@ -2171,7 +2170,7 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
         }
         ReadEntity viewInput = new ReadEntity(tab, parentInput, 
!qb.isInsideView());
         viewInput = PlanUtils.addInput(inputs, viewInput);
-        aliasToViewInfo.put(alias, new ObjectPair<String, 
ReadEntity>(fullViewName, viewInput));
+        aliasToViewInfo.put(alias, Pair.of(fullViewName, viewInput));
         String aliasId = getAliasId(alias, qb);
         if (aliasId != null) {
           aliasId = aliasId.replace(SemanticAnalyzer.SUBQUERY_TAG_1, "")
@@ -2221,8 +2220,8 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
       boolean wasCTE = sqAliasToCTEName.containsKey(alias);
       ReadEntity newParentInput = null;
       if (wasView) {
-        viewsExpanded.add(aliasToViewInfo.get(alias).getFirst());
-        newParentInput = aliasToViewInfo.get(alias).getSecond();
+        viewsExpanded.add(aliasToViewInfo.get(alias).getLeft());
+        newParentInput = aliasToViewInfo.get(alias).getRight();
       } else if (wasCTE) {
         ctesExpanded.add(sqAliasToCTEName.get(alias));
       }
@@ -4137,7 +4136,7 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
   // This function returns the grouping sets along with the grouping 
expressions
   // Even if rollups and cubes are present in the query, they are converted to
   // grouping sets at this point
-  ObjectPair<List<ASTNode>, List<Long>> getGroupByGroupingSetsForClause(
+  Pair<List<ASTNode>, List<Long>> getGroupByGroupingSetsForClause(
     QBParseInfo parseInfo, String dest) throws SemanticException {
     List<Long> groupingSets = new ArrayList<Long>();
     List<ASTNode> groupByExprs = getGroupByForClause(parseInfo, dest);
@@ -4154,7 +4153,7 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
       throw new 
SemanticException(ErrorMsg.HIVE_GROUPING_SETS_SIZE_LIMIT.getMsg());
     }
 
-    return new ObjectPair<List<ASTNode>, List<Long>>(groupByExprs, 
groupingSets);
+    return Pair.of(groupByExprs, groupingSets);
   }
 
   protected List<Long> getGroupingSets(List<ASTNode> groupByExpr, QBParseInfo 
parseInfo,
@@ -6115,11 +6114,10 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
     QBParseInfo parseInfo = qb.getParseInfo();
 
     int numReducers = -1;
-    ObjectPair<List<ASTNode>, List<Long>> grpByExprsGroupingSets =
-        getGroupByGroupingSetsForClause(parseInfo, dest);
+    Pair<List<ASTNode>, List<Long>> grpByExprsGroupingSets = 
getGroupByGroupingSetsForClause(parseInfo, dest);
 
-    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getFirst();
-    List<Long> groupingSets = grpByExprsGroupingSets.getSecond();
+    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getLeft();
+    List<Long> groupingSets = grpByExprsGroupingSets.getRight();
 
     if (grpByExprs.isEmpty()) {
       numReducers = 1;
@@ -6164,10 +6162,10 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
     List<ExprNodeDesc.ExprNodeDescEqualityWrapper> whereExpressions =
         new ArrayList<ExprNodeDesc.ExprNodeDescEqualityWrapper>();
     for (String dest : dests) {
-      ObjectPair<List<ASTNode>, List<Long>> grpByExprsGroupingSets =
+      Pair<List<ASTNode>, List<Long>> grpByExprsGroupingSets =
           getGroupByGroupingSetsForClause(parseInfo, dest);
 
-      List<Long> groupingSets = grpByExprsGroupingSets.getSecond();
+      List<Long> groupingSets = grpByExprsGroupingSets.getRight();
       if (!groupingSets.isEmpty()) {
         throw new 
SemanticException(ErrorMsg.HIVE_GROUPING_SETS_AGGR_NOMAPAGGR_MULTIGBY.getMsg());
       }
@@ -6301,11 +6299,10 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
 
     QBParseInfo parseInfo = qb.getParseInfo();
 
-    ObjectPair<List<ASTNode>, List<Long>> grpByExprsGroupingSets =
-        getGroupByGroupingSetsForClause(parseInfo, dest);
+    Pair<List<ASTNode>, List<Long>> grpByExprsGroupingSets = 
getGroupByGroupingSetsForClause(parseInfo, dest);
 
-    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getFirst();
-    List<Long> groupingSets = grpByExprsGroupingSets.getSecond();
+    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getLeft();
+    List<Long> groupingSets = grpByExprsGroupingSets.getRight();
 
     // Grouping sets are not allowed
     // This restriction can be lifted in future.
@@ -6496,11 +6493,10 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
                                                Operator inputOperatorInfo) 
throws SemanticException {
 
     QBParseInfo parseInfo = qb.getParseInfo();
-    ObjectPair<List<ASTNode>, List<Long>> grpByExprsGroupingSets =
-        getGroupByGroupingSetsForClause(parseInfo, dest);
+    Pair<List<ASTNode>, List<Long>> grpByExprsGroupingSets = 
getGroupByGroupingSetsForClause(parseInfo, dest);
 
-    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getFirst();
-    List<Long> groupingSets = grpByExprsGroupingSets.getSecond();
+    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getLeft();
+    List<Long> groupingSets = grpByExprsGroupingSets.getRight();
     boolean groupingSetsPresent = !groupingSets.isEmpty();
 
     int newMRJobGroupingSetsThreshold =
@@ -6665,11 +6661,10 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
 
     QBParseInfo parseInfo = qb.getParseInfo();
 
-    ObjectPair<List<ASTNode>, List<Long>> grpByExprsGroupingSets =
-        getGroupByGroupingSetsForClause(parseInfo, dest);
+    Pair<List<ASTNode>, List<Long>> grpByExprsGroupingSets = 
getGroupByGroupingSetsForClause(parseInfo, dest);
 
-    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getFirst();
-    List<Long> groupingSets = grpByExprsGroupingSets.getSecond();
+    List<ASTNode> grpByExprs = grpByExprsGroupingSets.getLeft();
+    List<Long> groupingSets = grpByExprsGroupingSets.getRight();
     boolean groupingSetsPresent = !groupingSets.isEmpty();
 
     if (groupingSetsPresent) {
@@ -10433,12 +10428,12 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
     }
   }
 
-  private ObjectPair<Integer, int[]> findMergePos(QBJoinTree node, QBJoinTree 
target) {
+  private Pair<Integer, int[]> findMergePos(QBJoinTree node, QBJoinTree 
target) {
     int res = -1;
     String leftAlias = node.getLeftAlias();
     if (leftAlias == null && (!node.getNoOuterJoin() || 
!target.getNoOuterJoin())) {
       // Cross with outer join: currently we do not merge
-      return new ObjectPair(-1, null);
+      return Pair.of(-1, null);
     }
 
     ArrayList<ASTNode> nodeCondn = node.getExpressions().get(0);
@@ -10458,7 +10453,7 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
     }
 
     if ( targetCondn == null || (nodeCondn.size() != targetCondn.size())) {
-      return new ObjectPair(-1, null);
+      return Pair.of(-1, null);
     }
 
     /*
@@ -10481,17 +10476,17 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
         }
       }
       if ( tgtToNodeExprMap[i] == -1) {
-        return new ObjectPair(-1, null);
+        return Pair.of(-1, null);
       }
     }
 
     for(j=0; j < nodeCondn.size(); j++) {
       if ( !nodeFiltersMapped[j]) {
-        return new ObjectPair(-1, null);
+        return Pair.of(-1, null);
       }
     }
 
-    return new ObjectPair(res, tgtToNodeExprMap);
+    return Pair.of(res, tgtToNodeExprMap);
   }
 
   boolean isCBOExecuted() {
@@ -10565,8 +10560,8 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
           // Outer joins with post-filtering conditions cannot be merged
           break;
         }
-        ObjectPair<Integer, int[]> mergeDetails = findMergePos(node, target);
-        int pos = mergeDetails.getFirst();
+        Pair<Integer, int[]> mergeDetails = findMergePos(node, target);
+        int pos = mergeDetails.getLeft();
         if (pos >= 0) {
           // for outer joins, it should not exceed 16 aliases (short type)
           if (!node.getNoOuterJoin() || !target.getNoOuterJoin()) {
@@ -10576,7 +10571,7 @@ public class SemanticAnalyzer extends 
BaseSemanticAnalyzer {
               continue;
             }
           }
-          mergeJoins(qb, node, target, pos, mergeDetails.getSecond());
+          mergeJoins(qb, node, target, pos, mergeDetails.getRight());
           trees.set(j, null);
           mergedQBJTree = true;
           continue; // continue merging with next alias
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
index ede9abc..e60f715 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.hive.ql.parse.spark;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
@@ -92,8 +92,7 @@ public class GenSparkProcContext implements NodeProcessorCtx {
 
   // map that keeps track of the last operator of a task to the following work
   // of this operator. This is used for connecting them later.
-  public final Map<ReduceSinkOperator, ObjectPair<SparkEdgeProperty, 
ReduceWork>>
-    leafOpToFollowingWorkInfo;
+  public final Map<ReduceSinkOperator, Pair<SparkEdgeProperty, ReduceWork>> 
leafOpToFollowingWorkInfo;
 
   // a map that keeps track of work that need to be linked while
   // traversing an operator tree
@@ -149,7 +148,6 @@ public class GenSparkProcContext implements 
NodeProcessorCtx {
   public final Set<Operator<?>> clonedPruningTableScanSet;
 
 
-  @SuppressWarnings("unchecked")
   public GenSparkProcContext(HiveConf conf,
       ParseContext parseContext,
       List<Task<MoveWork>> moveTask,
@@ -167,7 +165,7 @@ public class GenSparkProcContext implements 
NodeProcessorCtx {
     this.currentTask = SparkUtilities.createSparkTask(conf);
     this.rootTasks.add(currentTask);
     this.leafOpToFollowingWorkInfo =
-        new LinkedHashMap<ReduceSinkOperator, ObjectPair<SparkEdgeProperty, 
ReduceWork>>();
+        new LinkedHashMap<ReduceSinkOperator, Pair<SparkEdgeProperty, 
ReduceWork>>();
     this.linkOpWithWorkMap = new LinkedHashMap<Operator<?>, Map<BaseWork, 
SparkEdgeProperty>>();
     this.linkWorkWithReduceSinkMap = new LinkedHashMap<BaseWork, 
List<ReduceSinkOperator>>();
     this.smbMapJoinCtxMap = new HashMap<SMBMapJoinOperator, 
SparkSMBMapJoinInfo>();
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
index 04b1202..4fcc0c2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
@@ -27,19 +27,17 @@ import java.util.Stack;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
-import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
-import org.apache.hadoop.hive.ql.optimizer.spark.SparkSortMergeJoinFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.MapWork;
@@ -216,7 +214,7 @@ public class GenSparkWork implements NodeProcessor {
         rsOp.getConf().setOutputName(reduceWork.getName());
         GenMapRedUtils.setKeyAndValueDesc(reduceWork, rsOp);
 
-        context.leafOpToFollowingWorkInfo.put(rsOp, 
ObjectPair.create(edgeProp, reduceWork));
+        context.leafOpToFollowingWorkInfo.put(rsOp, Pair.of(edgeProp, 
reduceWork));
         LOG.debug("Removing " + parent + " as parent from " + root);
         root.removeParent(parent);
       }
@@ -242,10 +240,9 @@ public class GenSparkWork implements NodeProcessor {
     // Also note: the concept of leaf and root is reversed in hive for 
historical
     // reasons. Roots are data sources, leaves are data sinks. I know.
     if (context.leafOpToFollowingWorkInfo.containsKey(operator)) {
-      ObjectPair<SparkEdgeProperty, ReduceWork> childWorkInfo = context.
-        leafOpToFollowingWorkInfo.get(operator);
-      SparkEdgeProperty edgeProp = childWorkInfo.getFirst();
-      ReduceWork childWork = childWorkInfo.getSecond();
+      Pair<SparkEdgeProperty, ReduceWork> childWorkInfo = 
context.leafOpToFollowingWorkInfo.get(operator);
+      SparkEdgeProperty edgeProp = childWorkInfo.getLeft();
+      ReduceWork childWork = childWorkInfo.getRight();
 
       LOG.debug("Second pass. Leaf operator: " + operator + " has common 
downstream work:" + childWork);
 
diff --git 
a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
 
b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
index 251b31a..ef33054 100644
--- 
a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
+++ 
b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestReflectionObjectInspectors.java
@@ -29,10 +29,9 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang.mutable.MutableObject;
-import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantStringObjectInspector;
-import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaStringObjectInspector;
 import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.thrift.test.Complex;
 import com.google.common.collect.Lists;
@@ -146,23 +145,21 @@ public class TestReflectionObjectInspectors {
       @SuppressWarnings("unchecked")
       public void run() {
         Future<ObjectInspector>[] results = (Future<ObjectInspector>[])new 
Future[workerCount];
-        ObjectPair<Type, ObjectInspectorFactory.ObjectInspectorOptions>[] 
types =
-          (ObjectPair<Type, 
ObjectInspectorFactory.ObjectInspectorOptions>[])new ObjectPair[] {
-             new ObjectPair<Type, 
ObjectInspectorFactory.ObjectInspectorOptions>(Complex.class,
-               ObjectInspectorFactory.ObjectInspectorOptions.THRIFT),
-             new ObjectPair<Type, 
ObjectInspectorFactory.ObjectInspectorOptions>(MyStruct.class,
-               ObjectInspectorFactory.ObjectInspectorOptions.JAVA),
+        Pair<Type, ObjectInspectorFactory.ObjectInspectorOptions>[] types =
+          (Pair<Type, ObjectInspectorFactory.ObjectInspectorOptions>[])new 
Pair[] {
+             Pair.of(Complex.class, 
ObjectInspectorFactory.ObjectInspectorOptions.THRIFT),
+             Pair.of(MyStruct.class, 
ObjectInspectorFactory.ObjectInspectorOptions.JAVA),
           };
         try {
           for (int i = 0; i < 20; i++) { // repeat 20 times
-            for (final ObjectPair<Type, 
ObjectInspectorFactory.ObjectInspectorOptions> t: types) {
+            for (final Pair<Type, 
ObjectInspectorFactory.ObjectInspectorOptions> t: types) {
               ObjectInspectorFactory.objectInspectorCache.asMap().clear();
               for (int k = 0; k < workerCount; k++) {
                 results[k] = executorService.schedule(new 
Callable<ObjectInspector>() {
                   @Override
                   public ObjectInspector call() throws Exception {
                     return ObjectInspectorFactory.getReflectionObjectInspector(
-                      t.getFirst(), t.getSecond());
+                      t.getLeft(), t.getRight());
                   }
                 }, 50, TimeUnit.MILLISECONDS);
               }
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 12de51c..a906f61 100644
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -19,9 +19,7 @@
 package org.apache.hadoop.hive.metastore;
 
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
-import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.CAT_NAME;
 import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
-import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.parseDbName;
 import static 
org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.prependCatalogToDbName;
 
 import java.io.IOException;
@@ -54,6 +52,7 @@ import javax.security.auth.login.LoginException;
 
 import com.google.common.base.Preconditions;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -70,7 +69,6 @@ import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.utils.FilterUtils;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
 import org.apache.hadoop.hive.metastore.utils.LogUtils;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -319,7 +317,6 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 
     List<URI> metastoreURIArray = new ArrayList<URI>();
     try {
-      int i = 0;
       for (String s : metastoreUrisString) {
         URI tmpUri = new URI(s);
         if (tmpUri.getScheme() == null) {
@@ -1481,7 +1478,7 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-                                        List<ObjectPair<Integer, byte[]>> 
partExprs,
+                                        List<Pair<Integer, byte[]>> partExprs,
                                         PartitionDropOptions options)
       throws TException {
     return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs, 
options);
@@ -1489,7 +1486,7 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      List<Pair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResult) throws NoSuchObjectException, 
MetaException, TException {
 
     return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs,
@@ -1502,7 +1499,7 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      List<Pair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists) throws NoSuchObjectException, MetaException, 
TException {
     // By default, we need the results from dropPartitions();
     return dropPartitions(getDefaultCatalog(conf), dbName, tblName, partExprs,
@@ -1513,14 +1510,14 @@ public class HiveMetaStoreClient implements 
IMetaStoreClient, AutoCloseable {
 
   @Override
   public List<Partition> dropPartitions(String catName, String dbName, String 
tblName,
-                                        List<ObjectPair<Integer, byte[]>> 
partExprs,
+                                        List<Pair<Integer, byte[]>> partExprs,
                                         PartitionDropOptions options) throws 
TException {
     RequestPartsSpec rps = new RequestPartsSpec();
     List<DropPartitionsExpr> exprs = new ArrayList<>(partExprs.size());
-    for (ObjectPair<Integer, byte[]> partExpr : partExprs) {
+    for (Pair<Integer, byte[]> partExpr : partExprs) {
       DropPartitionsExpr dpe = new DropPartitionsExpr();
-      dpe.setExpr(partExpr.getSecond());
-      dpe.setPartArchiveLevel(partExpr.getFirst());
+      dpe.setExpr(partExpr.getRight());
+      dpe.setPartArchiveLevel(partExpr.getLeft());
       exprs.add(dpe);
     }
     rps.setExprs(exprs);
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 0129a89..6c7d80e 100644
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -34,7 +35,6 @@ import 
org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.annotation.NoReconnect;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
 
 /**
@@ -1892,7 +1892,7 @@ public interface IMetaStoreClient {
    * @throws TException Thrift transport error.
    */
   List<Partition> dropPartitions(String dbName, String tblName,
-                                 List<ObjectPair<Integer, byte[]>> partExprs, 
boolean deleteData,
+                                 List<Pair<Integer, byte[]>> partExprs, 
boolean deleteData,
                                  boolean ifExists) throws 
NoSuchObjectException, MetaException, TException;
 
   /**
@@ -1913,7 +1913,7 @@ public interface IMetaStoreClient {
    * @throws TException Thrift transport error.
    */
   default List<Partition> dropPartitions(String catName, String dbName, String 
tblName,
-                                         List<ObjectPair<Integer, byte[]>> 
partExprs,
+                                         List<Pair<Integer, byte[]>> partExprs,
                                          boolean deleteData, boolean ifExists)
       throws NoSuchObjectException, MetaException, TException {
     return dropPartitions(catName, dbName, tblName, partExprs,
@@ -1943,7 +1943,7 @@ public interface IMetaStoreClient {
    */
   @Deprecated
   List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      List<Pair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResults) throws NoSuchObjectException, 
MetaException, TException;
 
   /**
@@ -1966,7 +1966,7 @@ public interface IMetaStoreClient {
    * @throws TException Thrift transport error.
    */
   default List<Partition> dropPartitions(String catName, String dbName, String 
tblName,
-                                         List<ObjectPair<Integer, byte[]>> 
partExprs, boolean deleteData,
+                                         List<Pair<Integer, byte[]>> 
partExprs, boolean deleteData,
                                          boolean ifExists, boolean needResults)
       throws NoSuchObjectException, MetaException, TException {
     return dropPartitions(catName, dbName, tblName, partExprs,
@@ -1988,7 +1988,7 @@ public interface IMetaStoreClient {
    * @throws TException On failure
    */
   List<Partition> dropPartitions(String dbName, String tblName,
-                                 List<ObjectPair<Integer, byte[]>> partExprs,
+                                 List<Pair<Integer, byte[]>> partExprs,
                                  PartitionDropOptions options)
       throws NoSuchObjectException, MetaException, TException;
 
@@ -2005,7 +2005,7 @@ public interface IMetaStoreClient {
    * @throws TException On failure
    */
   List<Partition> dropPartitions(String catName, String dbName, String tblName,
-                                 List<ObjectPair<Integer, byte[]>> partExprs,
+                                 List<Pair<Integer, byte[]>> partExprs,
                                  PartitionDropOptions options)
       throws NoSuchObjectException, MetaException, TException;
 
diff --git 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/ObjectPair.java
 
b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/ObjectPair.java
deleted file mode 100644
index 5b49a25..0000000
--- 
a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/ObjectPair.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.metastore.utils;
-
-
-
-public class ObjectPair<F, S> {
-  private F first;
-  private S second;
-
-  public ObjectPair() {}
-
-  /**
-   * Creates a pair. Constructor doesn't infer template args but
-   * the method does, so the code becomes less ugly.
-   */
-  public static <T1, T2> ObjectPair<T1, T2> create(T1 f, T2 s) {
-    return new ObjectPair<>(f, s);
-  }
-
-  public ObjectPair(F first, S second) {
-    this.first = first;
-    this.second = second;
-  }
-
-  public F getFirst() {
-    return first;
-  }
-
-  public void setFirst(F first) {
-    this.first = first;
-  }
-
-  public S getSecond() {
-    return second;
-  }
-
-  public void setSecond(S second) {
-    this.second = second;
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null) {
-      return false;
-    }
-    if (that instanceof ObjectPair) {
-      return this.equals((ObjectPair<F, S>)that);
-    }
-    return false;
-  }
-
-  public boolean equals(ObjectPair<F, S> that) {
-    if (that == null) {
-      return false;
-    }
-
-    return this.getFirst().equals(that.getFirst()) &&
-        this.getSecond().equals(that.getSecond());
-  }
-
-  @Override
-  public int hashCode() {
-    return first.hashCode() * 31 + second.hashCode();
-  }
-
-  public String toString() {
-    return first + ":" + second;
-  }
-}
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Msck.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Msck.java
index d75d709..fab83b6 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Msck.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/Msck.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.hadoop.hive.metastore.utils.RetryUtilities;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.thrift.TException;
@@ -467,7 +467,7 @@ public class Msck {
             // so 3rd parameter (deleteData) is set to false
             // msck is doing a clean up of hms.  if for some reason the 
partition is already
             // deleted, then it is good.  So, the last parameter ifexists is 
set to true
-            List<ObjectPair<Integer, byte[]>> partExprs = 
getPartitionExpr(dropParts);
+            List<Pair<Integer, byte[]>> partExprs = 
getPartitionExpr(dropParts);
             metastoreClient.dropPartitions(table.getCatName(), 
table.getDbName(), table.getTableName(), partExprs, dropOptions);
 
             // if last batch is successful remove it from partsNotInFs
@@ -480,8 +480,8 @@ public class Msck {
         }
       }
 
-      private List<ObjectPair<Integer, byte[]>> getPartitionExpr(final 
List<String> parts) throws MetaException {
-        List<ObjectPair<Integer, byte[]>> expr = new ArrayList<>(parts.size());
+      private List<Pair<Integer, byte[]>> getPartitionExpr(final List<String> 
parts) throws MetaException {
+        List<Pair<Integer, byte[]>> expr = new ArrayList<>(parts.size());
         for (int i = 0; i < parts.size(); i++) {
           String partName = parts.get(i);
           Map<String, String> partSpec = Warehouse.makeSpecFromName(partName);
@@ -489,7 +489,7 @@ public class Msck {
           if (LOG.isDebugEnabled()) {
             LOG.debug("Generated partExpr: {} for partName: {}", partExpr, 
partName);
           }
-          expr.add(new ObjectPair<>(i, 
partExpr.getBytes(StandardCharsets.UTF_8)));
+          expr.add(Pair.of(i, partExpr.getBytes(StandardCharsets.UTF_8)));
         }
         return expr;
       }
diff --git 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index b40d816..ea6dc88 100644
--- 
a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ 
b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -66,6 +66,7 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configurable;
@@ -90,7 +91,6 @@ import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
 import org.datanucleus.store.rdbms.exceptions.MissingTableException;
 import org.slf4j.Logger;
@@ -134,7 +134,6 @@ public class ObjectStore implements RawStore, Configurable {
   public static final String TRUSTSTORE_PASSWORD_KEY = 
"javax.net.ssl.trustStorePassword";
   public static final String TRUSTSTORE_TYPE_KEY = 
"javax.net.ssl.trustStoreType";
 
-  private static final Map<String, Class<?>> PINCLASSMAP;
   private static final String HOSTNAME;
   private static final String USER;
   private static final String JDO_PARAM = ":param";
@@ -148,7 +147,6 @@ public class ObjectStore implements RawStore, Configurable {
     map.put("type", MType.class);
     map.put("fieldschema", MFieldSchema.class);
     map.put("order", MOrder.class);
-    PINCLASSMAP = Collections.unmodifiableMap(map);
     String hostname = "UNKNOWN";
     try {
       InetAddress clientAddr = InetAddress.getLocalHost();
@@ -1231,11 +1229,6 @@ public class ObjectStore implements RawStore, 
Configurable {
     return mConstraints;
   }
 
-  private static String getFullyQualifiedTableName(String dbName, String 
tblName) {
-    return ((dbName == null || dbName.isEmpty()) ? "" : "\"" + dbName + 
"\".\"")
-        + "\"" + tblName + "\"";
-  }
-
   @Override
   public Table
   getTable(String catName, String dbName, String tableName)
@@ -3339,7 +3332,6 @@ public class ObjectStore implements RawStore, 
Configurable {
       @Override
       protected List<Partition> getSqlResult(GetHelper<List<Partition>> ctx) 
throws MetaException {
         // If we have some sort of expression tree, try SQL filter pushdown.
-        List<Partition> result = null;
         if (exprTree != null) {
           SqlFilterForPushdown filter = new SqlFilterForPushdown();
           if (directSql.generateSqlFilterForPushdown(ctx.getTable(), exprTree, 
defaultPartitionName, filter)) {
@@ -3476,16 +3468,16 @@ public class ObjectStore implements RawStore, 
Configurable {
     return Batchable.runBatched(batchSize, partNames, new Batchable<String, 
Partition>() {
       @Override
       public List<Partition> run(List<String> input) throws MetaException {
-        ObjectPair<Query, Map<String, String>> queryWithParams =
+        Pair<Query, Map<String, String>> queryWithParams =
             getPartQueryWithParams(catName, dbName, tblName, input);
 
-        Query query = queryWithParams.getFirst();
+        Query query = queryWithParams.getLeft();
         query.setResultClass(MPartition.class);
         query.setClass(MPartition.class);
         query.setOrdering("partitionName ascending");
 
         @SuppressWarnings("unchecked")
-        List<MPartition> mparts = (List<MPartition>) 
query.executeWithMap(queryWithParams.getSecond());
+        List<MPartition> mparts = (List<MPartition>) 
query.executeWithMap(queryWithParams.getRight());
         List<Partition> partitions = convertToParts(catName, dbName, tblName, 
mparts);
         query.closeAll();
 
@@ -3495,11 +3487,11 @@ public class ObjectStore implements RawStore, 
Configurable {
   }
 
   private void dropPartitionsNoTxn(String catName, String dbName, String 
tblName, List<String> partNames) {
-    ObjectPair<Query, Map<String, String>> queryWithParams =
+    Pair<Query, Map<String, String>> queryWithParams =
         getPartQueryWithParams(catName, dbName, tblName, partNames);
-    Query query = queryWithParams.getFirst();
+    Query query = queryWithParams.getLeft();
     query.setClass(MPartition.class);
-    long deleted = query.deletePersistentAll(queryWithParams.getSecond());
+    long deleted = query.deletePersistentAll(queryWithParams.getRight());
     LOG.debug("Deleted {} partition from store", deleted);
     query.closeAll();
   }
@@ -3510,16 +3502,16 @@ public class ObjectStore implements RawStore, 
Configurable {
    * SDs; so, we remove the links to delete SDs and then check the returned 
CDs to see if
    * they are referenced by other SDs.
    */
-  private HashSet<MColumnDescriptor> detachCdsFromSdsNoTxn(
+  private Set<MColumnDescriptor> detachCdsFromSdsNoTxn(
       String catName, String dbName, String tblName, List<String> partNames) {
-    ObjectPair<Query, Map<String, String>> queryWithParams =
+    Pair<Query, Map<String, String>> queryWithParams =
         getPartQueryWithParams(catName, dbName, tblName, partNames);
-    Query query = queryWithParams.getFirst();
+    Query query = queryWithParams.getLeft();
     query.setClass(MPartition.class);
     query.setResult("sd");
     @SuppressWarnings("unchecked")
     List<MStorageDescriptor> sds = 
(List<MStorageDescriptor>)query.executeWithMap(
-        queryWithParams.getSecond());
+        queryWithParams.getRight());
     HashSet<MColumnDescriptor> candidateCds = new HashSet<>();
     for (MStorageDescriptor sd : sds) {
       if (sd != null && sd.getCD() != null) {
@@ -3527,9 +3519,7 @@ public class ObjectStore implements RawStore, 
Configurable {
         sd.setCD(null);
       }
     }
-    if (query != null) {
-      query.closeAll();
-    }
+    query.closeAll();
     return candidateCds;
   }
 
@@ -3568,7 +3558,7 @@ public class ObjectStore implements RawStore, 
Configurable {
     return queryFilter.toString();
   }
 
-  private ObjectPair<Query, Map<String, String>> getPartQueryWithParams(
+  private Pair<Query, Map<String, String>> getPartQueryWithParams(
       String catName, String dbName, String tblName, List<String> partNames) {
     Query query = pm.newQuery();
     Map<String, String> params = new HashMap<>();
@@ -3576,7 +3566,7 @@ public class ObjectStore implements RawStore, 
Configurable {
     query.setFilter(filterStr);
     LOG.debug(" JDOQL filter is {}", filterStr);
     query.declareParameters(makeParameterDeclarationString(params));
-    return new ObjectPair<>(query, params);
+    return Pair.of(query, params);
   }
 
   @Override
@@ -4204,7 +4194,6 @@ public class ObjectStore implements RawStore, 
Configurable {
   public Table alterTable(String catName, String dbname, String name, Table 
newTable,
       String queryValidWriteIds) throws InvalidObjectException, MetaException {
     boolean success = false;
-    boolean registerCreationSignature = false;
     try {
       openTransaction();
       name = normalizeIdentifier(name);
@@ -5154,7 +5143,6 @@ public class ObjectStore implements RawStore, 
Configurable {
       throws InvalidObjectException, MetaException {
     List<String> nnNames = new ArrayList<>();
     List<MConstraint> cstrs = new ArrayList<>();
-    String constraintName = null;
 
     for (int i = 0; i < cc.size(); i++) {
       final String catName = normalizeIdentifier(cc.get(i).getCatName());
@@ -5229,7 +5217,6 @@ public class ObjectStore implements RawStore, 
Configurable {
       throws InvalidObjectException, MetaException {
     List<String> nnNames = new ArrayList<>();
     List<MConstraint> cstrs = new ArrayList<>();
-    String constraintName = null;
 
     for (int i = 0; i < nns.size(); i++) {
       final String catName = normalizeIdentifier(nns.get(i).getCatName());
@@ -7052,11 +7039,11 @@ public class ObjectStore implements RawStore, 
Configurable {
 
   private void dropPartitionAllColumnGrantsNoTxn(
       String catName, String dbName, String tableName, List<String> partNames) 
{
-    ObjectPair<Query, Object[]> queryWithParams = 
makeQueryByPartitionNames(catName,
+    Pair<Query, Object[]> queryWithParams = makeQueryByPartitionNames(catName,
           dbName, tableName, partNames, MPartitionColumnPrivilege.class,
           "partition.table.tableName", "partition.table.database.name", 
"partition.partitionName",
           "partition.table.database.catalogName");
-    
queryWithParams.getFirst().deletePersistentAll(queryWithParams.getSecond());
+    queryWithParams.getLeft().deletePersistentAll(queryWithParams.getRight());
   }
 
   @SuppressWarnings("unchecked")
@@ -7121,27 +7108,27 @@ public class ObjectStore implements RawStore, 
Configurable {
 
   private void dropPartitionGrantsNoTxn(String catName, String dbName, String 
tableName,
                                         List<String> partNames) {
-    ObjectPair<Query, Object[]> queryWithParams = 
makeQueryByPartitionNames(catName,
+    Pair<Query, Object[]> queryWithParams = makeQueryByPartitionNames(catName,
           dbName, tableName, partNames,MPartitionPrivilege.class, 
"partition.table.tableName",
           "partition.table.database.name", "partition.partitionName",
           "partition.table.database.catalogName");
-    
queryWithParams.getFirst().deletePersistentAll(queryWithParams.getSecond());
+    queryWithParams.getLeft().deletePersistentAll(queryWithParams.getRight());
   }
 
   @SuppressWarnings("unchecked")
   private <T> List<T> queryByPartitionNames(String catName, String dbName, 
String tableName,
       List<String> partNames, Class<T> clazz, String tbCol, String dbCol, 
String partCol,
       String catCol) {
-    ObjectPair<Query, Object[]> queryAndParams = 
makeQueryByPartitionNames(catName,
+    Pair<Query, Object[]> queryAndParams = makeQueryByPartitionNames(catName,
         dbName, tableName, partNames, clazz, tbCol, dbCol, partCol, catCol);
-    return 
(List<T>)queryAndParams.getFirst().executeWithArray(queryAndParams.getSecond());
+    return 
(List<T>)queryAndParams.getLeft().executeWithArray(queryAndParams.getRight());
   }
 
-  private ObjectPair<Query, Object[]> makeQueryByPartitionNames(
+  private Pair<Query, Object[]> makeQueryByPartitionNames(
       String catName, String dbName, String tableName, List<String> partNames, 
Class<?> clazz,
       String tbCol, String dbCol, String partCol, String catCol) {
-    String queryStr = tbCol + " == t1 && " + dbCol + " == t2 && " + catCol + " 
== t3";
-    String paramStr = "java.lang.String t1, java.lang.String t2, 
java.lang.String t3";
+    StringBuilder queryStr = new StringBuilder(tbCol + " == t1 && " + dbCol + 
" == t2 && " + catCol + " == t3");
+    StringBuilder paramStr = new StringBuilder("java.lang.String t1, 
java.lang.String t2, java.lang.String t3");
     Object[] params = new Object[3 + partNames.size()];
     params[0] = normalizeIdentifier(tableName);
     params[1] = normalizeIdentifier(dbName);
@@ -7149,14 +7136,14 @@ public class ObjectStore implements RawStore, 
Configurable {
     int index = 0;
     for (String partName : partNames) {
       params[index + 3] = partName;
-      queryStr += ((index == 0) ? " && (" : " || ") + partCol + " == p" + 
index;
-      paramStr += ", java.lang.String p" + index;
+      queryStr.append(((index == 0) ? " && (" : " || ") + partCol + " == p" + 
index);
+      paramStr.append(", java.lang.String p" + index);
       ++index;
     }
-    queryStr += ")";
-    Query query = pm.newQuery(clazz, queryStr);
-    query.declareParameters(paramStr);
-    return new ObjectPair<>(query, params);
+    queryStr.append(")");
+    Query query = pm.newQuery(clazz, queryStr.toString());
+    query.declareParameters(paramStr.toString());
+    return Pair.of(query, params);
   }
 
   private List<MTablePrivilege> listAllMTableGrants(
@@ -9220,10 +9207,10 @@ public class ObjectStore implements RawStore, 
Configurable {
 
   private void dropPartitionColumnStatisticsNoTxn(
       String catName, String dbName, String tableName, List<String> partNames) 
throws MetaException {
-    ObjectPair<Query, Object[]> queryWithParams = makeQueryByPartitionNames(
+    Pair<Query, Object[]> queryWithParams = makeQueryByPartitionNames(
         catName, dbName, tableName, partNames, 
MPartitionColumnStatistics.class,
         "tableName", "dbName", "partition.partitionName", "catName");
-    
queryWithParams.getFirst().deletePersistentAll(queryWithParams.getSecond());
+    queryWithParams.getLeft().deletePersistentAll(queryWithParams.getRight());
   }
 
   @Override
diff --git 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 6aea508..afe7bc6 100644
--- 
a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ 
b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -52,6 +52,7 @@ import javax.security.auth.login.LoginException;
 
 import com.google.common.base.Preconditions;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -67,7 +68,6 @@ import 
org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
 import org.apache.hadoop.hive.metastore.txn.TxnCommonUtils;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -230,7 +230,6 @@ public class HiveMetaStoreClientPreCatalog implements 
IMetaStoreClient, AutoClos
 
     List<URI> metastoreURIArray = new ArrayList<URI>();
     try {
-      int i = 0;
       for (String s : metastoreUrisString) {
         URI tmpUri = new URI(s);
         if (tmpUri.getScheme() == null) {
@@ -257,7 +256,7 @@ public class HiveMetaStoreClientPreCatalog implements 
IMetaStoreClient, AutoClos
       }
 
       if (MetastoreConf.getVar(conf, 
ConfVars.THRIFT_URI_SELECTION).equalsIgnoreCase("RANDOM")) {
-        List uriList = Arrays.asList(metastoreUris);
+        List<URI> uriList = Arrays.asList(metastoreUris);
         Collections.shuffle(uriList);
         metastoreUris = (URI[]) uriList.toArray();
       }
@@ -1024,14 +1023,14 @@ public class HiveMetaStoreClientPreCatalog implements 
IMetaStoreClient, AutoClos
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-                                        List<ObjectPair<Integer, byte[]>> 
partExprs, PartitionDropOptions options)
+                                        List<Pair<Integer, byte[]>> partExprs, 
PartitionDropOptions options)
       throws TException {
     RequestPartsSpec rps = new RequestPartsSpec();
     List<DropPartitionsExpr> exprs = new ArrayList<>(partExprs.size());
-    for (ObjectPair<Integer, byte[]> partExpr : partExprs) {
+    for (Pair<Integer, byte[]> partExpr : partExprs) {
       DropPartitionsExpr dpe = new DropPartitionsExpr();
-      dpe.setExpr(partExpr.getSecond());
-      dpe.setPartArchiveLevel(partExpr.getFirst());
+      dpe.setExpr(partExpr.getRight());
+      dpe.setPartArchiveLevel(partExpr.getLeft());
       exprs.add(dpe);
     }
     rps.setExprs(exprs);
@@ -1048,7 +1047,7 @@ public class HiveMetaStoreClientPreCatalog implements 
IMetaStoreClient, AutoClos
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      List<Pair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResult) throws NoSuchObjectException, 
MetaException, TException {
 
     return dropPartitions(dbName, tblName, partExprs,
@@ -1061,7 +1060,7 @@ public class HiveMetaStoreClientPreCatalog implements 
IMetaStoreClient, AutoClos
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+      List<Pair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists) throws NoSuchObjectException, MetaException, 
TException {
     // By default, we need the results from dropPartitions();
     return dropPartitions(dbName, tblName, partExprs,
@@ -3373,7 +3372,7 @@ public class HiveMetaStoreClientPreCatalog implements 
IMetaStoreClient, AutoClos
 
   @Override
   public List<Partition> dropPartitions(String catName, String dbName, String 
tblName,
-                                        List<ObjectPair<Integer, byte[]>> 
partExprs,
+                                        List<Pair<Integer, byte[]>> partExprs,
                                         PartitionDropOptions options) throws 
NoSuchObjectException,
       MetaException, TException {
     throw new UnsupportedOperationException();

Reply via email to