[CARBONDATA-1395] Fix findbugs issues in carbondata-hadoop module

Fix findbugs issues in carbondata-hadoop module

This closes #1269
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
index e263aed..dde4c76 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
@@ -307,6 +307,27 @@
     return 0;
   }
 
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof CarbonInputSplit)) {
+      return false;
+    }
+    CarbonInputSplit other = (CarbonInputSplit) obj;
+    return 0 == this.compareTo(other);
+  }
+
+  @Override public int hashCode() {
+    int result = taskId.hashCode();
+    result = 31 * result + segmentId.hashCode();
+    result = 31 * result + bucketId.hashCode();
+    result = 31 * result + invalidSegments.hashCode();
+    result = 31 * result + numberOfBlocklets;
+    return result;
+  }
+
   @Override public String getBlockPath() {
     return getPath().getName();
   }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index dc4e76a..6ae346f 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -32,7 +32,6 @@
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
@@ -274,11 +273,6 @@
         invalidTimestampsList.add(updateStatusManager.getInvalidTimestampRange(invalidSegmentId));
       }
       if (invalidSegments.size() > 0) {
-        List<TableSegmentUniqueIdentifier> invalidSegmentsIds =
-            new ArrayList<>(invalidSegments.size());
-        for (String segId : invalidSegments) {
-          invalidSegmentsIds.add(new TableSegmentUniqueIdentifier(identifier, segId));
-        }
         blockletMap.clear(invalidSegments);
       }
     }
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputCommitter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputCommitter.java
deleted file mode 100644
index 50343c7..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableOutputCommitter.java
+++ /dev/null
@@ -1,55 +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.carbondata.hadoop.api;
-
-import java.io.IOException;
-
-import org.apache.carbondata.hadoop.internal.segment.Segment;
-import org.apache.carbondata.hadoop.internal.segment.SegmentManager;
-
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.JobContext;
-
-public class CarbonTableOutputCommitter extends FileOutputCommitter {
-  private SegmentManager segmentManager;
-  private Segment newSegment;
-
-  @Override
-  public void setupJob(JobContext context) throws IOException {
-    // steps:
-    // call segment manager to open new segment for loading
-    // load data synchronously
-    // close the segment and make it available for reading
-
-    newSegment = segmentManager.openNewSegment();
-    super.setupJob(context);
-  }
-
-  @Override
-  public void abortJob(JobContext context, int runState) throws IOException {
-    segmentManager.closeSegment(newSegment);
-    super.abortJob(context, runState);
-  }
-
-  @Override
-  public void commitJob(JobContext context) throws IOException {
-    newSegment.setupForRead(context);
-    segmentManager.commitSegment(newSegment);
-    super.commitJob(context);
-  }
-}
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java
index 8abc1f9..55b3973 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputSplitTaskInfo.java
@@ -67,6 +67,23 @@
     return taskId.compareTo(((CarbonInputSplitTaskInfo) o).getTaskId());
   }
 
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof CarbonInputSplitTaskInfo)) {
+      return false;
+    }
+
+    CarbonInputSplitTaskInfo that = (CarbonInputSplitTaskInfo)obj;
+    return null != taskId ? 0 == taskId.compareTo(that.taskId) : null == that.taskId;
+  }
+
+  @Override public int hashCode() {
+    return null != taskId ? taskId.hashCode() : 0;
+  }
+
   /**
    * Finding which node has the maximum number of blocks for it.
    *
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java
index 6266c82..59ac2f6 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java
@@ -18,8 +18,6 @@
 package org.apache.carbondata.spark.load;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -129,8 +127,6 @@
 
   public static boolean deleteLoadFoldersFromFileSystem(String dbName, String tableName,
       String storeLocation, boolean isForceDelete, LoadMetadataDetails[] details) {
-    List<LoadMetadataDetails> deletedLoads =
-        new ArrayList<LoadMetadataDetails>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
     boolean isDeleted = false;
 
@@ -142,7 +138,6 @@
           if (deletionStatus) {
             isDeleted = true;
             oneLoad.setVisibility("false");
-            deletedLoads.add(oneLoad);
             LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
           }
         }