You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by vb...@apache.org on 2019/09/11 18:08:34 UTC

[incubator-hudi] 01/02: [HUDI-159] Redesigning bundles for lighter-weight integrations

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

vbalaji pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hudi.git

commit 7a973a694488930627028b3175a731a23e3d3f32
Author: vinoth chandar <vc...@confluent.io>
AuthorDate: Mon Sep 2 16:15:55 2019 -0700

    [HUDI-159] Redesigning bundles for lighter-weight integrations
    
     - Documented principles applied for redesign at packaging/README.md
     - No longer depends on incl commons-codec, commons-io, commons-pool, commons-dbcp, commons-lang, commons-logging, avro-mapred
     - Introduce new FileIOUtils & added checkstyle rule for illegal import of above
     - Parquet, Avro dependencies moved to provided scope to enable being picked up from Hive/Spark/Presto instead
     - Pickup jackson jars for Hive sync tool from HIVE_HOME & unbundling jackson everywhere
     - Remove hive-jdbc standalone jar from being bundled in Spark/Hive/Utilities bundles
     - 6.5x reduced number of classes across bundles
---
 hudi-cli/pom.xml                                   |   6 -
 .../apache/hudi/cli/commands/RollbacksCommand.java |   2 +-
 .../java/org/apache/hudi/cli/utils/HiveUtil.java   |  16 +-
 .../java/org/apache/hudi/cli/utils/SparkUtil.java  |   4 +-
 .../BoundedPartitionAwareCompactionStrategy.java   |  18 +-
 .../main/java/org/apache/hudi/metrics/Metrics.java |   7 +-
 .../hudi/TestHoodieClientOnCopyOnWriteStorage.java |  11 +-
 .../org/apache/hudi/common/TestRawTripPayload.java |  10 +-
 .../apache/hudi/func/TestBoundedInMemoryQueue.java |   6 +-
 .../apache/hudi/func/TestUpdateMapFunction.java    |   4 +-
 .../hudi/index/bloom/TestHoodieBloomIndex.java     |   4 +-
 .../index/bloom/TestHoodieGlobalBloomIndex.java    |   4 +-
 .../io/strategy/TestHoodieCompactionStrategy.java  |  23 +-
 .../org/apache/hudi/metrics/TestHoodieMetrics.java |   3 +-
 .../apache/hudi/table/TestCopyOnWriteTable.java    |   4 +-
 hudi-common/pom.xml                                |  16 --
 .../java/org/apache/hudi/common/BloomFilter.java   |   2 +-
 .../org/apache/hudi/common/HoodieJsonPayload.java  |   7 +-
 .../table/timeline/HoodieActiveTimeline.java       |   4 +-
 .../table/timeline/HoodieDefaultTimeline.java      |   4 +-
 .../org/apache/hudi/common/util/AvroUtils.java     |  48 ----
 .../hudi/common/util/FailSafeConsistencyGuard.java |   2 +-
 .../org/apache/hudi/common/util/FileIOUtils.java   |  97 +++++++
 .../org/apache/hudi/common/util/RocksDBDAO.java    |   7 +-
 .../org/apache/hudi/common/util/StringUtils.java   |  11 +
 .../apache/hudi/common/util/collection/Pair.java   |  24 +-
 .../apache/hudi/common/util/collection/Triple.java |  21 +-
 .../hudi/common/minicluster/HdfsTestService.java   |  10 +-
 .../common/minicluster/ZookeeperTestService.java   |   6 +-
 .../apache/hudi/common/util/TestFileIOUtils.java   |  63 +++++
 hudi-hadoop-mr/pom.xml                             |   6 -
 .../org/apache/hudi/hadoop/HoodieInputFormat.java  |   6 +-
 .../hudi/hadoop/HoodieROTablePathFilter.java       |   6 +-
 .../hudi/hadoop/RecordReaderValueIterator.java     |   6 +-
 .../hadoop/hive/HoodieCombineHiveInputFormat.java  |   6 +-
 .../realtime/AbstractRealtimeRecordReader.java     |   7 +-
 .../hadoop/realtime/HoodieRealtimeInputFormat.java |   6 +-
 .../realtime/HoodieRealtimeRecordReader.java       |   6 +-
 .../realtime/RealtimeCompactedRecordReader.java    |   4 +
 hudi-hive/pom.xml                                  |  14 --
 hudi-hive/run_sync_tool.sh                         |   6 +-
 .../org/apache/hudi/hive/HoodieHiveClient.java     |  82 ++----
 .../java/org/apache/hudi/hive/util/SchemaUtil.java |   6 +-
 .../test/java/org/apache/hudi/hive/TestUtil.java   |   6 +-
 .../org/apache/hudi/hive/util/HiveTestService.java |   4 +-
 hudi-integ-test/pom.xml                            |   8 +-
 .../java/org/apache/hudi/integ/ITTestBase.java     |  21 +-
 .../org/apache/hudi/integ/ITTestHoodieDemo.java    |   2 +-
 hudi-utilities/pom.xml                             |   6 -
 .../hudi/utilities/HiveIncrementalPuller.java      |  24 +-
 .../deltastreamer/SchedulerConfGenerator.java      |  44 +---
 .../hudi/utilities/SchedulerConfGeneratorTest.java |  18 ++
 packaging/README.md                                |  18 ++
 packaging/hudi-hadoop-mr-bundle/pom.xml            | 238 ++++--------------
 packaging/hudi-hive-bundle/pom.xml                 | 221 ++++------------
 packaging/hudi-presto-bundle/pom.xml               | 212 ++++------------
 packaging/hudi-spark-bundle/pom.xml                | 270 ++++++--------------
 packaging/hudi-utilities-bundle/pom.xml            | 279 +++------------------
 pom.xml                                            |  70 +-----
 style/checkstyle.xml                               |   1 +
 60 files changed, 678 insertions(+), 1369 deletions(-)

diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml
index 9eb8876..819f902 100644
--- a/hudi-cli/pom.xml
+++ b/hudi-cli/pom.xml
@@ -176,12 +176,6 @@
       <artifactId>spark-sql_2.11</artifactId>
     </dependency>
 
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-dbcp</groupId>
-      <artifactId>commons-dbcp</artifactId>
-    </dependency>
-
     <dependency>
       <groupId>org.springframework.shell</groupId>
       <artifactId>spring-shell</artifactId>
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java
index 95fc730..bd568de 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.stream.Stream;
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
 import org.apache.hudi.cli.HoodieCLI;
 import org.apache.hudi.cli.HoodiePrintHelper;
@@ -37,6 +36,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieInstant.State;
 import org.apache.hudi.common.util.AvroUtils;
+import org.apache.hudi.common.util.collection.Pair;
 import org.springframework.shell.core.CommandMarker;
 import org.springframework.shell.core.annotation.CliCommand;
 import org.springframework.shell.core.annotation.CliOption;
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/HiveUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/HiveUtil.java
index 97fd85e..d0eaff8 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/HiveUtil.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/HiveUtil.java
@@ -19,11 +19,10 @@
 package org.apache.hudi.cli.utils;
 
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
-import javax.sql.DataSource;
-import org.apache.commons.dbcp.BasicDataSource;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.joda.time.DateTime;
 
@@ -42,17 +41,8 @@ public class HiveUtil {
   private static Connection connection;
 
   private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException {
-    DataSource ds = getDatasource(jdbcUrl, user, pass);
-    return ds.getConnection();
-  }
-
-  private static DataSource getDatasource(String jdbcUrl, String user, String pass) {
-    BasicDataSource ds = new BasicDataSource();
-    ds.setDriverClassName(driverName);
-    ds.setUrl(jdbcUrl);
-    ds.setUsername(user);
-    ds.setPassword(pass);
-    return ds;
+    connection = DriverManager.getConnection(jdbcUrl, user, pass);
+    return connection;
   }
 
   public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass)
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
index 69ac796..8dc2216 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
@@ -20,10 +20,10 @@ package org.apache.hudi.cli.utils;
 
 import java.io.File;
 import java.net.URISyntaxException;
-import org.apache.commons.lang.StringUtils;
 import org.apache.hudi.HoodieWriteClient;
 import org.apache.hudi.cli.commands.SparkMain;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaSparkContext;
@@ -43,7 +43,7 @@ public class SparkUtil {
     SparkLauncher sparkLauncher = new SparkLauncher().setAppResource(currentJar)
         .setMainClass(SparkMain.class.getName());
 
-    if (StringUtils.isNotEmpty(propertiesFile)) {
+    if (!StringUtils.isEmpty(propertiesFile)) {
       sparkLauncher.setPropertiesFile(propertiesFile);
     }
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/BoundedPartitionAwareCompactionStrategy.java b/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
index 6f0f793..7528730 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
+++ b/hudi-client/src/main/java/org/apache/hudi/io/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
@@ -18,13 +18,14 @@
 
 package org.apache.hudi.io.compact.strategy;
 
+import com.google.common.annotations.VisibleForTesting;
 import java.text.SimpleDateFormat;
+import java.util.Calendar;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
-import org.apache.commons.lang3.time.DateUtils;
 import org.apache.hudi.avro.model.HoodieCompactionOperation;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.config.HoodieWriteConfig;
@@ -45,8 +46,8 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
   public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
       List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
     // The earliest partition to compact - current day minus the target partitions limit
-    String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
-            .getTargetPartitionsPerDayBasedCompaction()));
+    String earliestPartitionPathToCompact = dateFormat.format(
+        getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
     // Filter out all partitions greater than earliestPartitionPathToCompact
     List<HoodieCompactionOperation> eligibleCompactionOperations = operations.stream()
         .collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
@@ -61,8 +62,8 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
   @Override
   public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
     // The earliest partition to compact - current day minus the target partitions limit
-    String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
-        .getTargetPartitionsPerDayBasedCompaction()));
+    String earliestPartitionPathToCompact = dateFormat.format(
+        getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
     // Get all partitions and sort them
     List<String> filteredPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
         .sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
@@ -70,4 +71,11 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
         .collect(Collectors.toList());
     return filteredPartitionPaths;
   }
+
+  @VisibleForTesting
+  public static Date getDateAtOffsetFromToday(int offset) {
+    Calendar calendar = Calendar.getInstance();
+    calendar.add(Calendar.DATE, offset);
+    return calendar.getTime();
+  }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
index 82219bb..fb970e1 100644
--- a/hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
+++ b/hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
@@ -22,7 +22,6 @@ import com.codahale.metrics.Gauge;
 import com.codahale.metrics.MetricRegistry;
 import com.google.common.io.Closeables;
 import java.io.Closeable;
-import org.apache.commons.configuration.ConfigurationException;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.log4j.LogManager;
@@ -37,9 +36,9 @@ public class Metrics {
   private static volatile boolean initialized = false;
   private static Metrics metrics = null;
   private final MetricRegistry registry;
-  private MetricsReporter reporter = null;
+  private MetricsReporter reporter;
 
-  private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
+  private Metrics(HoodieWriteConfig metricConfig) {
     registry = new MetricRegistry();
 
     reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
@@ -72,7 +71,7 @@ public class Metrics {
     }
     try {
       metrics = new Metrics(metricConfig);
-    } catch (ConfigurationException e) {
+    } catch (Exception e) {
       throw new HoodieException(e);
     }
     initialized = true;
diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java
index ed1458c..938ecae 100644
--- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java
+++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java
@@ -38,7 +38,6 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.HoodieClientTestUtils;
 import org.apache.hudi.common.HoodieTestDataGenerator;
@@ -55,6 +54,7 @@ import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.ConsistencyGuardConfig;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ParquetUtils;
 import org.apache.hudi.common.util.collection.Pair;
@@ -562,9 +562,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
     // Read from commit file
     String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
     FileInputStream inputStream = new FileInputStream(filename);
-    String everything = IOUtils.toString(inputStream, "UTF-8");
-    HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
-        HoodieCommitMetadata.class);
+    String everything = FileIOUtils.readAsUTFString(inputStream);
+    HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything, HoodieCommitMetadata.class);
     HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
     inputStream.close();
 
@@ -600,7 +599,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
     // Read from commit file
     String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
     FileInputStream inputStream = new FileInputStream(filename);
-    String everything = IOUtils.toString(inputStream, "UTF-8");
+    String everything = FileIOUtils.readAsUTFString(inputStream);
     HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
         HoodieCommitMetadata.class);
     HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
@@ -629,7 +628,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
     // Read from commit file
     filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
     inputStream = new FileInputStream(filename);
-    everything = IOUtils.toString(inputStream, "UTF-8");
+    everything = FileIOUtils.readAsUTFString(inputStream);
     metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class);
     rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
         .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
diff --git a/hudi-client/src/test/java/org/apache/hudi/common/TestRawTripPayload.java b/hudi-client/src/test/java/org/apache/hudi/common/TestRawTripPayload.java
index 32297cb..074aad4 100644
--- a/hudi-client/src/test/java/org/apache/hudi/common/TestRawTripPayload.java
+++ b/hudi-client/src/test/java/org/apache/hudi/common/TestRawTripPayload.java
@@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.StringWriter;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -32,11 +31,11 @@ import java.util.zip.DeflaterOutputStream;
 import java.util.zip.InflaterInputStream;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.IndexedRecord;
-import org.apache.commons.io.IOUtils;
 import org.apache.hudi.WriteStatus;
 import org.apache.hudi.avro.MercifulJsonConverter;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 
 /**
@@ -132,10 +131,9 @@ public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayloa
 
 
   private String unCompressData(byte[] data) throws IOException {
-    InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
-    StringWriter sw = new StringWriter(dataSize);
-    IOUtils.copy(iis, sw);
-    return sw.toString();
+    try (InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data))) {
+      return FileIOUtils.readAsUTFString(iis, dataSize);
+    }
   }
 
   /**
diff --git a/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java b/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java
index 043cc5e..a479137 100644
--- a/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java
+++ b/hudi-client/src/test/java/org/apache/hudi/func/TestBoundedInMemoryQueue.java
@@ -36,11 +36,11 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.avro.generic.IndexedRecord;
-import org.apache.commons.io.FileUtils;
 import org.apache.hudi.common.HoodieTestDataGenerator;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.util.DefaultSizeEstimator;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.SizeEstimator;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
@@ -82,7 +82,7 @@ public class TestBoundedInMemoryQueue {
     final int numRecords = 128;
     final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
     final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
-        new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
+        new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
     // Produce
     Future<Boolean> resFuture =
         executorService.submit(() -> {
@@ -122,7 +122,7 @@ public class TestBoundedInMemoryQueue {
     final List<List<HoodieRecord>> recs = new ArrayList<>();
 
     final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
-        new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
+        new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
 
     // Record Key to <Producer Index, Rec Index within a producer>
     Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
diff --git a/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java b/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java
index bfa1a49..a614c1c 100644
--- a/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java
+++ b/hudi-client/src/test/java/org/apache/hudi/func/TestUpdateMapFunction.java
@@ -27,7 +27,6 @@ import java.util.Arrays;
 import java.util.List;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.WriteStatus;
@@ -39,6 +38,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieTestUtils;
 import org.apache.hudi.common.table.HoodieTimeline;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.ParquetUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.io.HoodieCreateHandle;
@@ -151,7 +151,7 @@ public class TestUpdateMapFunction implements Serializable {
 
   private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
     // Prepare the AvroParquetIO
-    String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
+    String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream(schema));
     return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
   }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
index dffe3da..fad815f 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
+++ b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
@@ -37,7 +37,6 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import org.apache.avro.Schema;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.BloomFilter;
@@ -48,6 +47,7 @@ import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTestUtils;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.HoodieAvroUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
@@ -103,7 +103,7 @@ public class TestHoodieBloomIndex {
     fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
     HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
     // We have some records to be tagged (two different partitions)
-    schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
+    schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
     schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
   }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
index 90c2e0c..6d7df0d 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
+++ b/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
@@ -34,7 +34,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.avro.Schema;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hudi.common.HoodieClientTestUtils;
 import org.apache.hudi.common.TestRawTripPayload;
@@ -44,6 +43,7 @@ import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTestUtils;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.HoodieAvroUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
@@ -78,7 +78,7 @@ public class TestHoodieGlobalBloomIndex {
     fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
     HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
     // We have some records to be tagged (two different partitions)
-    schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
+    schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
     schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
   }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java
index f075edf..9a2f0c5 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java
+++ b/hudi-client/src/test/java/org/apache/hudi/io/strategy/TestHoodieCompactionStrategy.java
@@ -31,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.stream.Collectors;
-import org.apache.commons.lang3.time.DateUtils;
 import org.apache.hudi.avro.model.HoodieCompactionOperation;
 import org.apache.hudi.common.model.HoodieDataFile;
 import org.apache.hudi.common.model.HoodieLogFile;
@@ -161,11 +160,12 @@ public class TestHoodieCompactionStrategy {
     SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
     Date today = new Date();
     String currentDay = format.format(today);
-    String currentDayMinus1 = format.format(DateUtils.addDays(today, -1));
-    String currentDayMinus2 = format.format(DateUtils.addDays(today, -2));
-    String currentDayMinus3 = format.format(DateUtils.addDays(today, -3));
-    String currentDayPlus1 = format.format(DateUtils.addDays(today, 1));
-    String currentDayPlus5 = format.format(DateUtils.addDays(today, 5));
+
+    String currentDayMinus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-1));
+    String currentDayMinus2 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-2));
+    String currentDayMinus3 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-3));
+    String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
+    String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
 
     Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder()
         .put(120 * MB, currentDay)
@@ -208,11 +208,12 @@ public class TestHoodieCompactionStrategy {
     SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
     Date today = new Date();
     String currentDay = format.format(today);
-    String currentDayMinus1 = format.format(DateUtils.addDays(today, -1));
-    String currentDayMinus2 = format.format(DateUtils.addDays(today, -2));
-    String currentDayMinus3 = format.format(DateUtils.addDays(today, -3));
-    String currentDayPlus1 = format.format(DateUtils.addDays(today, 1));
-    String currentDayPlus5 = format.format(DateUtils.addDays(today, 5));
+
+    String currentDayMinus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-1));
+    String currentDayMinus2 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-2));
+    String currentDayMinus3 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-3));
+    String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
+    String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
 
     Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder()
         .put(120 * MB, currentDay)
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java b/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java
index 1a1b407..a99c614 100644
--- a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java
+++ b/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import org.apache.commons.configuration.ConfigurationException;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.junit.Before;
 import org.junit.Test;
@@ -33,7 +32,7 @@ public class TestHoodieMetrics {
   private HoodieMetrics metrics = null;
 
   @Before
-  public void start() throws ConfigurationException {
+  public void start() {
     HoodieWriteConfig config = mock(HoodieWriteConfig.class);
     when(config.isMetricsOn()).thenReturn(true);
     when(config.getMetricsReporterType()).thenReturn(MetricsReporterType.INMEMORY);
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java b/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java
index 980c158..bdbf094 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java
+++ b/hudi-client/src/test/java/org/apache/hudi/table/TestCopyOnWriteTable.java
@@ -30,7 +30,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.WriteStatus;
 import org.apache.hudi.common.BloomFilter;
@@ -45,6 +44,7 @@ import org.apache.hudi.common.model.HoodieTestUtils;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.HoodieTimeline;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ParquetUtils;
 import org.apache.hudi.common.util.collection.Pair;
@@ -117,7 +117,7 @@ public class TestCopyOnWriteTable {
 
   private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
     // Prepare the AvroParquetIO
-    String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
+    String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
     return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
   }
 
diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml
index db3ffe9..2b83af7 100644
--- a/hudi-common/pom.xml
+++ b/hudi-common/pom.xml
@@ -94,16 +94,6 @@
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-mapred</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
 
     <!-- Parquet -->
     <dependency>
@@ -111,12 +101,6 @@
       <artifactId>parquet-avro</artifactId>
     </dependency>
 
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-codec</groupId>
-      <artifactId>commons-codec</artifactId>
-    </dependency>
-
     <!-- Httpcomponents -->
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/BloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/BloomFilter.java
index 5b18ef7..57ab2d8 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/BloomFilter.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/BloomFilter.java
@@ -19,12 +19,12 @@
 package org.apache.hudi.common;
 
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import javax.xml.bind.DatatypeConverter;
-import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.hadoop.util.bloom.Key;
 import org.apache.hadoop.util.hash.Hash;
 import org.apache.hudi.exception.HoodieIndexException;
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/HoodieJsonPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/HoodieJsonPayload.java
index 9f3eb93..0675653 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/HoodieJsonPayload.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/HoodieJsonPayload.java
@@ -23,15 +23,14 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.StringWriter;
 import java.util.zip.Deflater;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.InflaterInputStream;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.IndexedRecord;
-import org.apache.commons.io.IOUtils;
 import org.apache.hudi.avro.MercifulJsonConverter;
 import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieException;
 
@@ -87,9 +86,7 @@ public class HoodieJsonPayload implements HoodieRecordPayload<HoodieJsonPayload>
   private String unCompressData(byte[] data) throws IOException {
     InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
     try {
-      StringWriter sw = new StringWriter(dataSize);
-      IOUtils.copy(iis, sw);
-      return sw.toString();
+      return FileIOUtils.readAsUTFString(iis, dataSize);
     } finally {
       iis.close();
     }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
index c86c3a1..fd62eef 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
@@ -30,13 +30,13 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Stream;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.HoodieTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant.State;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.log4j.LogManager;
@@ -394,7 +394,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
 
   private Option<byte[]> readDataFromPath(Path detailPath) {
     try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
-      return Option.of(IOUtils.toByteArray(is));
+      return Option.of(FileIOUtils.readAsByteArray(is));
     } catch (IOException e) {
       throw new HoodieIOException("Could not read commit details from " + detailPath, e);
     }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
index 9f8154f..acf9bfd 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
@@ -27,7 +27,6 @@ import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import org.apache.commons.codec.binary.Hex;
 import org.apache.hudi.common.table.HoodieTimeline;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
@@ -68,7 +67,8 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
     } catch (NoSuchAlgorithmException nse) {
       throw new HoodieException(nse);
     }
-    this.timelineHash = new String(Hex.encodeHex(md.digest()));
+
+    this.timelineHash = StringUtils.toHexString(md.digest());
   }
 
   /**
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/AvroUtils.java
index 01baa47..b0a01b1 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/AvroUtils.java
@@ -20,28 +20,20 @@ package org.apache.hudi.common.util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileReader;
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.file.FileReader;
 import org.apache.avro.file.SeekableByteArrayInput;
-import org.apache.avro.file.SeekableInput;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.io.DatumReader;
 import org.apache.avro.io.DatumWriter;
-import org.apache.avro.mapred.FsInput;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.avro.specific.SpecificRecordBase;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.model.HoodieCleanMetadata;
 import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
@@ -52,49 +44,9 @@ import org.apache.hudi.avro.model.HoodieSavepointMetadata;
 import org.apache.hudi.avro.model.HoodieSavepointPartitionMetadata;
 import org.apache.hudi.common.HoodieCleanStat;
 import org.apache.hudi.common.HoodieRollbackStat;
-import org.apache.hudi.common.model.HoodieAvroPayload;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.exception.HoodieIOException;
 
 public class AvroUtils {
 
-  public static List<HoodieRecord<HoodieAvroPayload>> loadFromFiles(FileSystem fs,
-      List<String> deltaFilePaths, Schema expectedSchema) {
-    List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
-    deltaFilePaths.forEach(s -> {
-      List<HoodieRecord<HoodieAvroPayload>> records = loadFromFile(fs, s, expectedSchema);
-      loadedRecords.addAll(records);
-    });
-    return loadedRecords;
-  }
-
-  public static List<HoodieRecord<HoodieAvroPayload>> loadFromFile(FileSystem fs,
-      String deltaFilePath, Schema expectedSchema) {
-    List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
-    Path path = new Path(deltaFilePath);
-    try {
-      SeekableInput input = new FsInput(path, fs.getConf());
-      GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>();
-      // Set the expected schema to be the current schema to account for schema evolution
-      reader.setExpected(expectedSchema);
-
-      FileReader<GenericRecord> fileReader = DataFileReader.openReader(input, reader);
-      for (GenericRecord deltaRecord : fileReader) {
-        String key = deltaRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
-        String partitionPath =
-            deltaRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
-        loadedRecords.add(new HoodieRecord<>(new HoodieKey(key, partitionPath),
-            new HoodieAvroPayload(Option.of(deltaRecord))));
-      }
-      fileReader.close(); // also closes underlying FsInput
-    } catch (IOException e) {
-      throw new HoodieIOException("Could not read avro records from path " + deltaFilePath,
-          e);
-    }
-    return loadedRecords;
-  }
-
   public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime,
       Option<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
     ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FailSafeConsistencyGuard.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FailSafeConsistencyGuard.java
index e1b816e..bec07a8 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/FailSafeConsistencyGuard.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FailSafeConsistencyGuard.java
@@ -107,7 +107,7 @@ public class FailSafeConsistencyGuard implements ConsistencyGuard {
         log.warn("Got IOException waiting for file event. Have tried " + retryNum + " time(s)", ioe);
       }
       return false;
-    }, "Timed out waiting for filles to become visible");
+    }, "Timed out waiting for files to become visible");
   }
 
   /**
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java
new file mode 100644
index 0000000..908c780
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FileIOUtils.java
@@ -0,0 +1,97 @@
+/*
+ * 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.hudi.common.util;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Comparator;
+
+/**
+ * Bunch of utility methods for working with files and byte streams
+ */
+public class FileIOUtils {
+
+  public static final long KB = 1024;
+
+  public static void deleteDirectory(File directory) throws IOException {
+    if (directory.exists()) {
+      Files.walk(directory.toPath())
+          .sorted(Comparator.reverseOrder())
+          .map(Path::toFile)
+          .forEach(File::delete);
+      directory.delete();
+      if (directory.exists()) {
+        throw new IOException("Unable to delete directory " + directory);
+      }
+    }
+  }
+
+  public static void mkdir(File directory) throws IOException {
+    if (!directory.exists()) {
+      directory.mkdirs();
+    }
+
+    if (!directory.isDirectory()) {
+      throw new IOException("Unable to create :" + directory);
+    }
+  }
+
+  public static String readAsUTFString(InputStream input) throws IOException {
+    return readAsUTFString(input, 128);
+  }
+
+  public static String readAsUTFString(InputStream input, int length) throws IOException {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(length);
+    copy(input, bos);
+    return new String(bos.toByteArray(), StandardCharsets.UTF_8);
+  }
+
+  public static void copy(InputStream inputStream, OutputStream outputStream) throws IOException {
+    byte[] buffer = new byte[1024];
+    int len;
+    while ((len = inputStream.read(buffer)) != -1) {
+      outputStream.write(buffer, 0, len);
+    }
+  }
+
+  public static byte[] readAsByteArray(InputStream input) throws IOException {
+    return readAsByteArray(input, 128);
+  }
+
+  public static byte[] readAsByteArray(InputStream input, int outputSize) throws IOException {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream(outputSize);
+    copy(input, bos);
+    return bos.toByteArray();
+  }
+
+  public static void writeStringToFile(String str, String filePath) throws IOException {
+    PrintStream out = new PrintStream(new FileOutputStream(filePath));
+    out.println(str);
+    out.flush();
+    out.close();
+  }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBDAO.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBDAO.java
index 52992a2..2ae53f9 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBDAO.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBDAO.java
@@ -30,7 +30,6 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import org.apache.commons.io.FileUtils;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
@@ -86,7 +85,7 @@ public class RocksDBDAO {
   private void init() throws HoodieException {
     try {
       log.info("DELETING RocksDB persisted at " + rocksDBBasePath);
-      FileUtils.deleteDirectory(new File(rocksDBBasePath));
+      FileIOUtils.deleteDirectory(new File(rocksDBBasePath));
 
       managedHandlesMap = new ConcurrentHashMap<>();
       managedDescriptorMap = new ConcurrentHashMap<>();
@@ -103,7 +102,7 @@ public class RocksDBDAO {
       });
       final List<ColumnFamilyDescriptor> managedColumnFamilies = loadManagedColumnFamilies(dbOptions);
       final List<ColumnFamilyHandle> managedHandles = new ArrayList<>();
-      FileUtils.forceMkdir(new File(rocksDBBasePath));
+      FileIOUtils.mkdir(new File(rocksDBBasePath));
       rocksDB = RocksDB.open(dbOptions, rocksDBBasePath, managedColumnFamilies, managedHandles);
 
       Preconditions.checkArgument(managedHandles.size() == managedColumnFamilies.size(),
@@ -450,7 +449,7 @@ public class RocksDBDAO {
       managedDescriptorMap.clear();
       getRocksDB().close();
       try {
-        FileUtils.deleteDirectory(new File(rocksDBBasePath));
+        FileIOUtils.deleteDirectory(new File(rocksDBBasePath));
       } catch (IOException e) {
         throw new HoodieIOException(e.getMessage(), e);
       }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java
index d3d162d..5fd45b8 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java
@@ -54,4 +54,15 @@ public class StringUtils {
     return org.apache.hadoop.util.StringUtils.join(separator, array);
   }
 
+  public static String toHexString(byte[] bytes) {
+    StringBuilder sb = new StringBuilder(bytes.length * 2);
+    for (byte b: bytes) {
+      sb.append(String.format("%02x", b));
+    }
+    return sb.toString();
+  }
+
+  public static boolean isEmpty(String str) {
+    return str == null || str.length() == 0;
+  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Pair.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Pair.java
index 28fcc98..bcd800c 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Pair.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Pair.java
@@ -20,7 +20,6 @@ package org.apache.hudi.common.util.collection;
 
 import java.io.Serializable;
 import java.util.Map;
-import org.apache.commons.lang.builder.CompareToBuilder;
 
 /**
  * (NOTE: Adapted from Apache commons-lang3)
@@ -57,7 +56,7 @@ public abstract class Pair<L, R> implements Map.Entry<L, R>, Comparable<Pair<L,
    * @return a pair formed from the two parameters, not null
    */
   public static <L, R> Pair<L, R> of(final L left, final R right) {
-    return new ImmutablePair<L, R>(left, right);
+    return new ImmutablePair<>(left, right);
   }
 
   //-----------------------------------------------------------------------
@@ -117,8 +116,20 @@ public abstract class Pair<L, R> implements Map.Entry<L, R>, Comparable<Pair<L,
    */
   @Override
   public int compareTo(final Pair<L, R> other) {
-    return new CompareToBuilder().append(getLeft(), other.getLeft())
-        .append(getRight(), other.getRight()).toComparison();
+
+    checkComparable(this);
+    checkComparable(other);
+
+    Comparable thisLeft = (Comparable) getLeft();
+    Comparable thisRight = (Comparable) getRight();
+    Comparable otherLeft = (Comparable) other.getLeft();
+    Comparable otherRight = (Comparable) other.getRight();
+
+    if (thisLeft.compareTo(otherLeft) == 0) {
+      return thisRight.compareTo(otherRight);
+    } else {
+      return thisLeft.compareTo(otherLeft);
+    }
   }
 
   /**
@@ -178,4 +189,9 @@ public abstract class Pair<L, R> implements Map.Entry<L, R>, Comparable<Pair<L,
     return String.format(format, getLeft(), getRight());
   }
 
+  private void checkComparable(Pair<L, R> pair) {
+    if (!(pair.getLeft() instanceof Comparable) || !(pair.getRight() instanceof Comparable)) {
+      throw new IllegalArgumentException("Elements of Pair must implement Comparable :" + pair);
+    }
+  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Triple.java b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Triple.java
index 8a030e1..55bb63b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Triple.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/collection/Triple.java
@@ -19,7 +19,6 @@
 package org.apache.hudi.common.util.collection;
 
 import java.io.Serializable;
-import org.apache.commons.lang.builder.CompareToBuilder;
 
 /**
  * (NOTE: Adapted from Apache commons-lang3)
@@ -96,9 +95,17 @@ public abstract class Triple<L, M, R> implements Comparable<Triple<L, M, R>>, Se
    */
   @Override
   public int compareTo(final Triple<L, M, R> other) {
-    return new CompareToBuilder().append(getLeft(), other.getLeft())
-        .append(getMiddle(), other.getMiddle())
-        .append(getRight(), other.getRight()).toComparison();
+    checkComparable(this);
+    checkComparable(other);
+
+    Comparable thisLeft = (Comparable) getLeft();
+    Comparable otherLeft = (Comparable) other.getLeft();
+
+    if (thisLeft.compareTo(otherLeft) == 0) {
+      return Pair.of(getMiddle(), getRight()).compareTo(Pair.of(other.getMiddle(), other.getRight()));
+    } else {
+      return thisLeft.compareTo(otherLeft);
+    }
   }
 
   /**
@@ -160,5 +167,11 @@ public abstract class Triple<L, M, R> implements Comparable<Triple<L, M, R>>, Se
     return String.format(format, getLeft(), getMiddle(), getRight());
   }
 
+  private void checkComparable(Triple<L, M, R> triplet) {
+    if (!(triplet.getLeft() instanceof Comparable) || !(triplet.getMiddle() instanceof Comparable)
+        || !(triplet.getRight() instanceof Comparable)) {
+      throw new IllegalArgumentException("Elements of Triple must implement Comparable :" + triplet);
+    }
+  }
 }
 
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java
index 5408189..09eaa73 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/minicluster/HdfsTestService.java
@@ -22,21 +22,21 @@ import com.google.common.base.Preconditions;
 import com.google.common.io.Files;
 import java.io.File;
 import java.io.IOException;
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hudi.common.model.HoodieTestUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * An HDFS minicluster service implementation.
  */
 public class HdfsTestService {
 
-  private static final Logger logger = LoggerFactory.getLogger(HdfsTestService.class);
+  private static final Logger logger = LogManager.getLogger(HdfsTestService.class);
 
   /**
    * Configuration settings
@@ -72,7 +72,7 @@ public class HdfsTestService {
     if (format) {
       logger.info("Cleaning HDFS cluster data at: " + localDFSLocation + " and starting fresh.");
       File file = new File(localDFSLocation);
-      FileUtils.deleteDirectory(file);
+      FileIOUtils.deleteDirectory(file);
     }
 
     // Configure and start the HDFS cluster
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/minicluster/ZookeeperTestService.java b/hudi-common/src/test/java/org/apache/hudi/common/minicluster/ZookeeperTestService.java
index 1a01012..89992ee 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/minicluster/ZookeeperTestService.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/minicluster/ZookeeperTestService.java
@@ -30,11 +30,11 @@ import java.net.InetSocketAddress;
 import java.net.Socket;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 import org.apache.zookeeper.server.NIOServerCnxnFactory;
 import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.persistence.FileTxnLog;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A Zookeeper minicluster service implementation.
@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
  */
 public class ZookeeperTestService {
 
-  private static final Logger logger = LoggerFactory.getLogger(ZookeeperTestService.class);
+  private static final Logger logger = LogManager.getLogger(ZookeeperTestService.class);
 
   private static final int TICK_TIME = 2000;
   private static final int CONNECTION_TIMEOUT = 30000;
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestFileIOUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestFileIOUtils.java
new file mode 100644
index 0000000..222c7aa
--- /dev/null
+++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestFileIOUtils.java
@@ -0,0 +1,63 @@
+/*
+ * 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.hudi.common.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestFileIOUtils {
+
+  @Test
+  public void testMkdirAndDelete() throws IOException {
+    TemporaryFolder folder = new TemporaryFolder();
+    folder.create();
+    try {
+      FileIOUtils.mkdir(folder.getRoot());
+    } catch (IOException e) {
+      fail("Should not error out if dir exists already");
+    }
+    File dir = new File(folder.getRoot().getAbsolutePath() + "/dir");
+    FileIOUtils.mkdir(dir);
+    assertTrue(dir.exists());
+
+    new File(dir, "t.txt").createNewFile();
+    new File(dir, "subdir").mkdirs();
+    new File(dir, "subdir/z.txt").createNewFile();
+    FileIOUtils.deleteDirectory(dir);
+    assertFalse(dir.exists());
+  }
+
+  @Test
+  public void testInputStreamReads() throws IOException {
+    String msg = "hudi rocks!";
+    ByteArrayInputStream inputStream = new ByteArrayInputStream(msg.getBytes(StandardCharsets.UTF_8));
+    assertEquals(msg, FileIOUtils.readAsUTFString(inputStream));
+    inputStream = new ByteArrayInputStream(msg.getBytes(StandardCharsets.UTF_8));
+    assertEquals(msg.length(), FileIOUtils.readAsByteArray(inputStream).length);
+  }
+}
diff --git a/hudi-hadoop-mr/pom.xml b/hudi-hadoop-mr/pom.xml
index 25574e7..07487df 100644
--- a/hudi-hadoop-mr/pom.xml
+++ b/hudi-hadoop-mr/pom.xml
@@ -49,12 +49,6 @@
       <artifactId>parquet-avro</artifactId>
     </dependency>
 
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
-
     <!-- Hadoop -->
     <dependency>
       <groupId>org.apache.hadoop</groupId>
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieInputFormat.java
index 9f8e4d5..a12579c 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieInputFormat.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieInputFormat.java
@@ -24,8 +24,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -49,6 +47,8 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
 import org.apache.hudi.exception.DatasetNotFoundException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.InvalidDatasetException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * HoodieInputFormat which understands the Hoodie File Structure and filters files based on the
@@ -59,7 +59,7 @@ import org.apache.hudi.exception.InvalidDatasetException;
 @UseFileSplitsFromInputFormat
 public class HoodieInputFormat extends MapredParquetInputFormat implements Configurable {
 
-  public static final Log LOG = LogFactory.getLog(HoodieInputFormat.class);
+  private static final transient Logger LOG = LogManager.getLogger(HoodieInputFormat.class);
 
   protected Configuration conf;
 
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java
index f268810..aaac779 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java
@@ -23,8 +23,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.stream.Collectors;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,6 +33,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
 import org.apache.hudi.exception.DatasetNotFoundException;
 import org.apache.hudi.exception.HoodieException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * Given a path is a part of - Hoodie dataset = accepts ONLY the latest version of each path -
@@ -48,7 +48,7 @@ import org.apache.hudi.exception.HoodieException;
  */
 public class HoodieROTablePathFilter implements PathFilter, Serializable {
 
-  public static final Log LOG = LogFactory.getLog(HoodieROTablePathFilter.class);
+  private static final transient Logger LOG = LogManager.getLogger(HoodieROTablePathFilter.class);
 
   /**
    * Its quite common, to have all files from a given partition path be passed into accept(), cache
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java
index f8c231a..d68afb4 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RecordReaderValueIterator.java
@@ -21,10 +21,10 @@ package org.apache.hudi.hadoop;
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hudi.exception.HoodieException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * Provides Iterator Interface to iterate value entries read from record reader
@@ -34,7 +34,7 @@ import org.apache.hudi.exception.HoodieException;
  */
 public class RecordReaderValueIterator<K, V> implements Iterator<V> {
 
-  public static final Log LOG = LogFactory.getLog(RecordReaderValueIterator.class);
+  private static final transient Logger LOG = LogManager.getLogger(RecordReaderValueIterator.class);
 
   private final RecordReader<K, V> reader;
   private V nextVal = null;
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
index 06ae8d6..501b0c4 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
@@ -72,8 +72,8 @@ import org.apache.hadoop.mapred.lib.CombineFileSplit;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hudi.hadoop.HoodieInputFormat;
 import org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * This is just a copy of the org.apache.hadoop.hive.ql.io.CombineHiveInputFormat from Hive 2.x
@@ -92,7 +92,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
     extends HiveInputFormat<K, V> {
 
   private static final String CLASS_NAME = HoodieCombineHiveInputFormat.class.getName();
-  public static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+  public static final Logger LOG = LogManager.getLogger(CLASS_NAME);
 
   // max number of threads we can use to check non-combinable paths
   private static final int MAX_CHECK_NONCOMBINABLE_THREAD_NUM = 50;
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java
index a82b0bc..93a608c 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/AbstractRealtimeRecordReader.java
@@ -31,8 +31,6 @@ import org.apache.avro.Schema.Field;
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
@@ -53,6 +51,8 @@ import org.apache.hudi.common.util.LogReaderUtils;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 import org.apache.parquet.avro.AvroSchemaConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.schema.MessageType;
@@ -82,7 +82,8 @@ public abstract class AbstractRealtimeRecordReader {
   // Default file path prefix for spillable file
   public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/";
 
-  public static final Log LOG = LogFactory.getLog(AbstractRealtimeRecordReader.class);
+  private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class);
+
   protected final HoodieRealtimeFileSplit split;
   protected final JobConf jobConf;
   private final MessageType baseFileSchema;
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeInputFormat.java
index 67f3e46..bd354e0 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeInputFormat.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeInputFormat.java
@@ -29,8 +29,6 @@ import java.util.Map;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -56,6 +54,8 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.hadoop.HoodieInputFormat;
 import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * Input Format, that provides a real-time view of data in a Hoodie dataset
@@ -63,7 +63,7 @@ import org.apache.hudi.hadoop.UseFileSplitsFromInputFormat;
 @UseFileSplitsFromInputFormat
 public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Configurable {
 
-  public static final Log LOG = LogFactory.getLog(HoodieRealtimeInputFormat.class);
+  private static final transient Logger LOG = LogManager.getLogger(HoodieRealtimeInputFormat.class);
 
   // These positions have to be deterministic across all tables
   public static final int HOODIE_COMMIT_TIME_COL_POS = 0;
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java
index b1fab44..5e095d8 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeRecordReader.java
@@ -19,13 +19,13 @@
 package org.apache.hudi.hadoop.realtime;
 
 import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hudi.exception.HoodieException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * Realtime Record Reader which can do compacted (merge-on-read) record reading or
@@ -37,7 +37,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, Ar
   public static final String REALTIME_SKIP_MERGE_PROP = "hoodie.realtime.merge.skip";
   // By default, we do merged-reading
   public static final String DEFAULT_REALTIME_SKIP_MERGE = "false";
-  public static final Log LOG = LogFactory.getLog(HoodieRealtimeRecordReader.class);
+  private static final transient Logger LOG = LogManager.getLogger(HoodieRealtimeRecordReader.class);
   private final RecordReader<NullWritable, ArrayWritable> reader;
 
   public HoodieRealtimeRecordReader(HoodieRealtimeFileSplit split, JobConf job,
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
index 8f22352..9c54b56 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
@@ -32,10 +32,14 @@ import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.util.FSUtils;
 import org.apache.hudi.common.util.HoodieAvroUtils;
 import org.apache.hudi.common.util.Option;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader implements
     RecordReader<NullWritable, ArrayWritable> {
 
+  private static final Logger LOG = LogManager.getLogger(AbstractRealtimeRecordReader.class);
+
   protected final RecordReader<NullWritable, ArrayWritable> parquetReader;
   private final Map<String, HoodieRecord<? extends HoodieRecordPayload>> deltaRecordMap;
 
diff --git a/hudi-hive/pom.xml b/hudi-hive/pom.xml
index 8fdcf58..af6c827 100644
--- a/hudi-hive/pom.xml
+++ b/hudi-hive/pom.xml
@@ -65,20 +65,6 @@
       <artifactId>joda-time</artifactId>
     </dependency>
 
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-dbcp</groupId>
-      <artifactId>commons-dbcp</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-pool</groupId>
-      <artifactId>commons-pool</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-
     <dependency>
       <groupId>com.beust</groupId>
       <artifactId>jcommander</artifactId>
diff --git a/hudi-hive/run_sync_tool.sh b/hudi-hive/run_sync_tool.sh
index cd58db3..60cc661 100755
--- a/hudi-hive/run_sync_tool.sh
+++ b/hudi-hive/run_sync_tool.sh
@@ -44,12 +44,12 @@ fi
 HIVE_EXEC=`ls ${HIVE_HOME}/lib/hive-exec-*.jar | tr '\n' ':'`
 HIVE_SERVICE=`ls ${HIVE_HOME}/lib/hive-service-*.jar | grep -v rpc | tr '\n' ':'`
 HIVE_METASTORE=`ls ${HIVE_HOME}/lib/hive-metastore-*.jar | tr '\n' ':'`
-# Hive 1.x/CDH has standalone jdbc jar which is no longer available in 2.x
-HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*standalone*.jar | tr '\n' ':'`
+HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*.jar | tr '\n' ':'`
 if [ -z "${HIVE_JDBC}" ]; then
   HIVE_JDBC=`ls ${HIVE_HOME}/lib/hive-jdbc-*.jar | grep -v handler | tr '\n' ':'`
 fi
-HIVE_JARS=$HIVE_METASTORE:$HIVE_SERVICE:$HIVE_EXEC:$HIVE_SERVICE:$HIVE_JDBC
+HIVE_JACKSON=`ls ${HIVE_HOME}/lib/jackson-*.jar | tr '\n' ':'`
+HIVE_JARS=$HIVE_METASTORE:$HIVE_SERVICE:$HIVE_EXEC:$HIVE_SERVICE:$HIVE_JDBC:$HIVE_JACKSON
 
 HADOOP_HIVE_JARS=${HIVE_JARS}:${HADOOP_HOME}/share/hadoop/common/*:${HADOOP_HOME}/share/hadoop/mapreduce/*:${HADOOP_HOME}/share/hadoop/hdfs/*:${HADOOP_HOME}/share/hadoop/common/lib/*:${HADOOP_HOME}/share/hadoop/hdfs/lib/*
 
diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
index 4a47801..92667d8 100644
--- a/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
+++ b/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
@@ -24,7 +24,7 @@ import com.google.common.collect.Maps;
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
-import java.sql.Driver;
+import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -33,9 +33,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
-import org.apache.commons.dbcp.BasicDataSource;
-import org.apache.commons.dbcp.ConnectionFactory;
-import org.apache.commons.dbcp.DriverConnectionFactory;
+import jline.internal.Log;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -57,13 +55,13 @@ import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.InvalidDatasetException;
 import org.apache.hudi.hive.util.SchemaUtil;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.schema.MessageType;
 import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 @SuppressWarnings("ConstantConditions")
 public class HoodieHiveClient {
@@ -80,7 +78,7 @@ public class HoodieHiveClient {
     }
   }
 
-  private static Logger LOG = LoggerFactory.getLogger(HoodieHiveClient.class);
+  private static Logger LOG = LogManager.getLogger(HoodieHiveClient.class);
   private final HoodieTableMetaClient metaClient;
   private final HoodieTableType tableType;
   private final PartitionValueExtractor partitionValueExtractor;
@@ -473,20 +471,18 @@ public class HoodieHiveClient {
 
   private void createHiveConnection() {
     if (connection == null) {
-      BasicDataSource ds = new HiveDataSource();
-      ds.setDriverClassName(HiveDriver.class.getCanonicalName());
-      ds.setUrl(getHiveJdbcUrlWithDefaultDBName());
-      if (syncConfig.hiveUser != null) {
-        ds.setUsername(syncConfig.hiveUser);
-        ds.setPassword(syncConfig.hivePass);
+      try {
+        Class.forName(HiveDriver.class.getCanonicalName());
+      } catch (ClassNotFoundException e) {
+        Log.error("Unable to load Hive driver class", e);
+        return;
       }
-      LOG.info("Getting Hive Connection from Datasource " + ds);
+
       try {
-        this.connection = ds.getConnection();
-        LOG.info("Successfully got Hive Connection from Datasource " + ds);
+        this.connection = DriverManager.getConnection(syncConfig.jdbcUrl, syncConfig.hiveUser, syncConfig.hivePass);
+        LOG.info("Successfully established Hive connection to  " + syncConfig.jdbcUrl);
       } catch (SQLException e) {
-        throw new HoodieHiveSyncException(
-            "Cannot create hive connection " + getHiveJdbcUrlWithDefaultDBName(), e);
+        throw new HoodieHiveSyncException("Cannot create hive connection " + getHiveJdbcUrlWithDefaultDBName(), e);
       }
     }
   }
@@ -627,54 +623,4 @@ public class HoodieHiveClient {
       return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
     }
   }
-
-  /**
-   * There is a bug in BasicDataSource implementation (dbcp-1.4) which does not allow custom version of Driver (needed
-   * to talk to older version of HiveServer2 including CDH-5x). This is fixed in dbcp-2x but we are using dbcp1.4.
-   * Adding a workaround here. TODO: varadarb We need to investigate moving to dbcp-2x
-   */
-  protected class HiveDataSource extends BasicDataSource {
-
-    protected ConnectionFactory createConnectionFactory() throws SQLException {
-      try {
-        Driver driver = HiveDriver.class.newInstance();
-        // Can't test without a validationQuery
-        if (validationQuery == null) {
-          setTestOnBorrow(false);
-          setTestOnReturn(false);
-          setTestWhileIdle(false);
-        }
-
-        // Set up the driver connection factory we will use
-        String user = username;
-        if (user != null) {
-          connectionProperties.put("user", user);
-        } else {
-          log("DBCP DataSource configured without a 'username'");
-        }
-
-        String pwd = password;
-        if (pwd != null) {
-          connectionProperties.put("password", pwd);
-        } else {
-          log("DBCP DataSource configured without a 'password'");
-        }
-
-        ConnectionFactory driverConnectionFactory = new DriverConnectionFactory(driver, url, connectionProperties);
-        return driverConnectionFactory;
-      } catch (Throwable x) {
-        LOG.warn("Got exception trying to instantiate connection factory. Trying default instantiation", x);
-        return super.createConnectionFactory();
-      }
-    }
-
-    @Override
-    public String toString() {
-      return "HiveDataSource{"
-          + "driverClassName='" + driverClassName + '\''
-          + ", driverClassLoader=" + driverClassLoader
-          + ", url='" + url + '\''
-          + '}';
-    }
-  }
 }
diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java b/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java
index 9e2784e..6cd7747 100644
--- a/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java
+++ b/hudi-hive/src/main/java/org/apache/hudi/hive/util/SchemaUtil.java
@@ -36,6 +36,8 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.hive.HiveSyncConfig;
 import org.apache.hudi.hive.HoodieHiveSyncException;
 import org.apache.hudi.hive.SchemaDifference;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 import org.apache.parquet.avro.AvroSchemaConverter;
 import org.apache.parquet.schema.DecimalMetadata;
 import org.apache.parquet.schema.GroupType;
@@ -43,15 +45,13 @@ import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Schema Utilities
  */
 public class SchemaUtil {
 
-  private static final Logger LOG = LoggerFactory.getLogger(SchemaUtil.class);
+  private static final Logger LOG = LogManager.getLogger(SchemaUtil.class);
 
   /**
    * Get the schema difference between the storage schema and hive table schema
diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java b/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java
index 6e9e06b..f49b7df 100644
--- a/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java
+++ b/hudi-hive/src/test/java/org/apache/hudi/hive/TestUtil.java
@@ -34,7 +34,6 @@ import java.util.Set;
 import java.util.UUID;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.IndexedRecord;
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -61,6 +60,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
 import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.SchemaTestUtil;
 import org.apache.hudi.hive.util.HiveTestService;
 import org.apache.parquet.avro.AvroSchemaConverter;
@@ -153,7 +153,7 @@ public class TestUtil {
   static void createCOWDataset(String commitTime, int numberOfPartitions)
       throws IOException, InitializationError, URISyntaxException, InterruptedException {
     Path path = new Path(hiveSyncConfig.basePath);
-    FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
+    FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
     HoodieTableMetaClient
         .initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
             hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
@@ -169,7 +169,7 @@ public class TestUtil {
   static void createMORDataset(String commitTime, String deltaCommitTime, int numberOfPartitions)
       throws IOException, InitializationError, URISyntaxException, InterruptedException {
     Path path = new Path(hiveSyncConfig.basePath);
-    FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
+    FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
     HoodieTableMetaClient
         .initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ,
             hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java b/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
index 487200e..8961558 100644
--- a/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
+++ b/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
@@ -28,7 +28,6 @@ import java.net.SocketException;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -41,6 +40,7 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.thrift.TUGIContainingTransport;
 import org.apache.hive.service.server.HiveServer2;
 import org.apache.hudi.common.model.HoodieTestUtils;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TProcessor;
@@ -95,7 +95,7 @@ public class HiveTestService {
     if (clean) {
       LOG.info("Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh.");
       File file = new File(localHiveLocation);
-      FileUtils.deleteDirectory(file);
+      FileIOUtils.deleteDirectory(file);
     }
 
     HiveConf serverConf = configureHive(hadoopConf, localHiveLocation);
diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml
index ffe16e1..e562369 100644
--- a/hudi-integ-test/pom.xml
+++ b/hudi-integ-test/pom.xml
@@ -42,12 +42,8 @@
 
     <!-- Logging -->
     <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
     </dependency>
 
     <!-- Hoodie - Import -->
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java
index c12cf6d..9b33ba3 100644
--- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestBase.java
@@ -38,6 +38,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -215,6 +216,19 @@ public abstract class ITTestBase {
     return Pair.of(callback.getStdout().toString(), callback.getStderr().toString());
   }
 
+  private void saveUpLogs() {
+    try {
+      // save up the Hive log files for introspection
+      String hiveLogStr = executeCommandStringInDocker(HIVESERVER, "cat /tmp/root/hive.log", true)
+          .getStdout().toString();
+      String filePath = System.getProperty("java.io.tmpdir") + "/" + System.currentTimeMillis() + "-hive.log";
+      FileIOUtils.writeStringToFile(hiveLogStr, filePath);
+      LOG.info("Hive log saved up at  : " + filePath);
+    } catch (Exception e) {
+      LOG.error("Unable to save up logs..", e);
+    }
+  }
+
   void assertStdOutContains(Pair<String, String> stdOutErr, String expectedOutput) {
     assertStdOutContains(stdOutErr, expectedOutput, 1);
   }
@@ -233,7 +247,12 @@ public abstract class ITTestBase {
         lastIndex += expectedOutput.length();
       }
     }
-   Assert.assertEquals("Did not find output the expected number of times", times, count);
+
+    if (times != count) {
+      saveUpLogs();
+    }
+
+    Assert.assertEquals("Did not find output the expected number of times", times, count);
   }
 
   public class TestExecStartResultCallback extends ExecStartResultCallback {
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java
index ee3855a..1779625 100644
--- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/ITTestHoodieDemo.java
@@ -19,8 +19,8 @@
 package org.apache.hudi.integ;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.hudi.common.util.collection.Pair;
 import java.util.List;
+import org.apache.hudi.common.util.collection.Pair;
 import org.junit.Test;
 
 /**
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index ce83276..bf5d104 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -138,12 +138,6 @@
       <artifactId>jackson-module-scala_2.11</artifactId>
     </dependency>
 
-    <!-- Avro -->
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-mapred</artifactId>
-    </dependency>
-
     <!-- Parquet -->
     <dependency>
       <groupId>org.apache.parquet</groupId>
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java
index 465d235..3f443c3 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java
@@ -25,15 +25,13 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.Serializable;
 import java.sql.Connection;
+import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.List;
 import java.util.Scanner;
 import java.util.stream.Collectors;
-import javax.sql.DataSource;
-import org.apache.commons.dbcp.BasicDataSource;
-import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -41,6 +39,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.utilities.exception.HoodieIncrementalPullException;
@@ -110,8 +109,8 @@ public class HiveIncrementalPuller {
   public HiveIncrementalPuller(Config config) throws IOException {
     this.config = config;
     validateConfig(config);
-    String templateContent = IOUtils.toString(
-        this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"), "UTF-8");
+    String templateContent = FileIOUtils.readAsUTFString(
+        this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
     incrementalPullSQLtemplate = new ST(templateContent);
   }
 
@@ -344,22 +343,13 @@ public class HiveIncrementalPuller {
 
   private Connection getConnection() throws SQLException {
     if (connection == null) {
-      DataSource ds = getDatasource();
-      log.info("Getting Hive Connection from Datasource " + ds);
-      this.connection = ds.getConnection();
+      log.info("Getting Hive Connection to " + config.hiveJDBCUrl);
+      this.connection = DriverManager.getConnection(config.hiveJDBCUrl, config.hiveUsername, config.hivePassword);
+
     }
     return connection;
   }
 
-  private DataSource getDatasource() {
-    BasicDataSource ds = new BasicDataSource();
-    ds.setDriverClassName(driverName);
-    ds.setUrl(config.hiveJDBCUrl);
-    ds.setUsername(config.hiveUsername);
-    ds.setPassword(config.hivePassword);
-    return ds;
-  }
-
   public static void main(String[] args) throws IOException {
     final Config cfg = new Config();
     JCommander cmd = new JCommander(cfg, args);
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java
index 418df27..8f0a6b9 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SchedulerConfGenerator.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.commons.lang.text.StrSubstitutor;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.util.Option;
 import org.apache.log4j.LogManager;
@@ -44,47 +43,26 @@ public class SchedulerConfGenerator {
   public static final String SPARK_SCHEDULER_MODE_KEY = "spark.scheduler.mode";
   public static final String SPARK_SCHEDULER_ALLOCATION_FILE_KEY = "spark.scheduler.allocation.file";
 
-
-  private static final String DELTASYNC_POOL_KEY = "deltasync_pool";
-  private static final String COMPACT_POOL_KEY = "compact_pool";
-  private static final String DELTASYNC_POLICY_KEY = "deltasync_policy";
-  private static final String COMPACT_POLICY_KEY = "compact_policy";
-  private static final String DELTASYNC_WEIGHT_KEY = "deltasync_weight";
-  private static final String DELTASYNC_MINSHARE_KEY = "deltasync_minshare";
-  private static final String COMPACT_WEIGHT_KEY = "compact_weight";
-  private static final String COMPACT_MINSHARE_KEY = "compact_minshare";
-
   private static String SPARK_SCHEDULING_PATTERN =
       "<?xml version=\"1.0\"?>\n"
           + "<allocations>\n"
-          + "  <pool name=\"%(deltasync_pool)\">\n"
-          + "    <schedulingMode>%(deltasync_policy)</schedulingMode>\n"
-          + "    <weight>%(deltasync_weight)</weight>\n"
-          + "    <minShare>%(deltasync_minshare)</minShare>\n"
+          + "  <pool name=\"%s\">\n"
+          + "    <schedulingMode>%s</schedulingMode>\n"
+          + "    <weight>%s</weight>\n"
+          + "    <minShare>%s</minShare>\n"
           + "  </pool>\n"
-          + "  <pool name=\"%(compact_pool)\">\n"
-          + "    <schedulingMode>%(compact_policy)</schedulingMode>\n"
-          + "    <weight>%(compact_weight)</weight>\n"
-          + "    <minShare>%(compact_minshare)</minShare>\n"
+          + "  <pool name=\"%s\">\n"
+          + "    <schedulingMode>%s</schedulingMode>\n"
+          + "    <weight>%s</weight>\n"
+          + "    <minShare>%s</minShare>\n"
           + "  </pool>\n"
           + "</allocations>";
 
   private static String generateConfig(Integer deltaSyncWeight, Integer compactionWeight, Integer deltaSyncMinShare,
       Integer compactionMinShare) {
-    Map<String, String> schedulingProps = new HashMap<>();
-    schedulingProps.put(DELTASYNC_POOL_KEY, DELTASYNC_POOL_NAME);
-    schedulingProps.put(COMPACT_POOL_KEY, COMPACT_POOL_NAME);
-    schedulingProps.put(DELTASYNC_POLICY_KEY, "FAIR");
-    schedulingProps.put(COMPACT_POLICY_KEY, "FAIR");
-    schedulingProps.put(DELTASYNC_WEIGHT_KEY, deltaSyncWeight.toString());
-    schedulingProps.put(DELTASYNC_MINSHARE_KEY, deltaSyncMinShare.toString());
-    schedulingProps.put(COMPACT_WEIGHT_KEY, compactionWeight.toString());
-    schedulingProps.put(COMPACT_MINSHARE_KEY, compactionMinShare.toString());
-
-    StrSubstitutor sub = new StrSubstitutor(schedulingProps, "%(", ")");
-    String xmlString = sub.replace(SPARK_SCHEDULING_PATTERN);
-    log.info("Scheduling Configurations generated. Config=\n" + xmlString);
-    return xmlString;
+    return String.format(SPARK_SCHEDULING_PATTERN,
+        DELTASYNC_POOL_NAME, "FAIR", deltaSyncWeight.toString(), deltaSyncMinShare.toString(),
+        COMPACT_POOL_NAME, "FAIR", compactionWeight.toString(), compactionMinShare.toString());
   }
 
 
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/SchedulerConfGeneratorTest.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/SchedulerConfGeneratorTest.java
index 091c436..7a249c9 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/SchedulerConfGeneratorTest.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/SchedulerConfGeneratorTest.java
@@ -1,3 +1,21 @@
+/*
+ * 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.hudi.utilities;
 
 import static org.junit.Assert.assertNotNull;
diff --git a/packaging/README.md b/packaging/README.md
new file mode 100644
index 0000000..d2b112d
--- /dev/null
+++ b/packaging/README.md
@@ -0,0 +1,18 @@
+# Overview
+
+This folder contains several modules that build out bundles (i.e fat/uber jars) that enable hudi integration into various systems.
+
+Here are the key principles applied in designing these bundles
+
+ - As much as possible, try to make the bundle work with the target system's jars and classes. (e.g: better to make Hudi work with Hive's parquet version than bundling parquet with Hudi). This lets us evolve Hudi as a lighter weight component and also provides flexibility for changing these jar versions in target systems
+ - Bundle's pom only needs to depend on the required hudi modules & any other modules that are declared "provided" in parent poms (e.g: parquet-avro). 
+ - Such other modules should be declared as "compile" dependency in the bundle pom to actually get the shade plugin in pull them into the bundle. By default, provided scoped dependencies are not included
+ - Any other runtime dependencies needed by the bundle should specified in the `<include>` whitelist. New bundles also should follow the same style of explicitly whitelisting modules and shading as needed.
+ - Leave abundant comments on why someone is being included, shaded or even being left out.
+
+Please follow these when adding new ones or making changes.
+
+# Resources 
+
+ 1. Classes needed for Hive2 JDBC documented [here](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveServer2Clients-RunningtheJDBCSampleCode)
+ 
\ No newline at end of file
diff --git a/packaging/hudi-hadoop-mr-bundle/pom.xml b/packaging/hudi-hadoop-mr-bundle/pom.xml
index 81b5e3d..be37d04 100644
--- a/packaging/hudi-hadoop-mr-bundle/pom.xml
+++ b/packaging/hudi-hadoop-mr-bundle/pom.xml
@@ -23,141 +23,16 @@
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
-
   <artifactId>hudi-hadoop-mr-bundle</artifactId>
   <packaging>jar</packaging>
 
-  <dependencies>
-    <!-- Hoodie -->
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-hadoop-mr</artifactId>
-      <version>${project.version}</version>
-      <exclusions>
-        <exclusion>
-          <!-- other hoodie deps will come from hoodie-hive-bundle -->
-          <groupId>org.apache.hudi</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <!-- Avro -->
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-
-    <!-- Parquet -->
-    <dependency>
-      <groupId>org.apache.parquet</groupId>
-      <artifactId>parquet-avro</artifactId>
-    </dependency>
-
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-codec</groupId>
-      <artifactId>commons-codec</artifactId>
-    </dependency>
-
-    <!-- Hadoop -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-auth</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-    </dependency>
-
-    <!-- Hive -->
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-jdbc</artifactId>
-      <version>${hive.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>commons-logging</groupId>
-          <artifactId>commons-logging</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-exec</artifactId>
-      <version>${hive.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-service</artifactId>
-      <version>${hive.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-shims</artifactId>
-      <version>${hive.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-serde</artifactId>
-      <version>${hive.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-metastore</artifactId>
-      <version>${hive.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-common</artifactId>
-      <version>${hive.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.esotericsoftware</groupId>
-      <artifactId>kryo</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-  </dependencies>
+  <properties>
+    <checkstyle.skip>true</checkstyle.skip>
+    <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
+    <notice.file>HUDI_NOTICE.txt</notice.file>
+  </properties>
 
   <build>
-    <resources>
-      <resource>
-        <directory>src/main/resources</directory>
-      </resource>
-    </resources>
     <plugins>
       <plugin>
         <groupId>org.apache.rat</groupId>
@@ -166,7 +41,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
-        <version>2.4</version>
+        <version>${maven-shade-plugin.version}</version>
         <executions>
           <execution>
             <phase>package</phase>
@@ -177,65 +52,33 @@
               <createSourcesJar>true</createSourcesJar>
               <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml
               </dependencyReducedPomLocation>
-              <relocations>
-                 <relocation>
-                   <pattern>parquet.avro</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.avro</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.column</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.column</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.format.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.format.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.hadoop.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.hadoop.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.schema</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.schema</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.apache.commons.</pattern>
-                   <shadedPattern>org.apache.hudi.org.apache.commons.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.apache.commons.codec.</pattern>
-                   <shadedPattern>org.apache.hudi.org.apache.commons.codec.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>com.esotericsoftware.kryo.</pattern>
-                   <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.objenesis.</pattern>
-                   <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>com.esotericsoftware.minlog.</pattern>
-                   <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
-                 </relocation>
-              </relocations>
-              <createDependencyReducedPom>false</createDependencyReducedPom>
               <artifactSet>
                 <includes>
                   <include>org.apache.hudi:hudi-common</include>
                   <include>org.apache.hudi:hudi-hadoop-mr</include>
-                  <include>commons-logging:commons-logging</include>
-                  <include>commons-io:commons-io</include>
-                  <include>commons-lang:commons-lang</include>
-                  <include>commons-pool:commons-pool</include>
-                  <include>commons-codec:commons-codec</include>
+
+                  <include>org.apache.parquet:parquet-avro</include>
                   <include>com.esotericsoftware:kryo-shaded</include>
                   <include>org.objenesis:objenesis</include>
                   <include>com.esotericsoftware:minlog</include>
-                  <include>commons-codec:commons-codec</include>
-                  <include>org.apache.parquet:parquet-avro</include>
                 </includes>
               </artifactSet>
+              <relocations>
+
+                <relocation>
+                  <pattern>com.esotericsoftware.kryo.</pattern>
+                  <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.objenesis.</pattern>
+                  <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.esotericsoftware.minlog.</pattern>
+                  <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
+                </relocation>
+              </relocations>
+              <createDependencyReducedPom>false</createDependencyReducedPom>
               <filters>
                 <filter>
                   <artifact>*:*</artifact>
@@ -255,11 +98,34 @@
         </executions>
       </plugin>
     </plugins>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+      </resource>
+      <resource>
+        <directory>src/test/resources</directory>
+      </resource>
+    </resources>
   </build>
 
-  <properties>
-    <checkstyle.skip>true</checkstyle.skip>
-    <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
-    <notice.file>HUDI_NOTICE.txt</notice.file>
-  </properties>
+  <dependencies>
+    <!-- Hoodie -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hadoop-mr</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- Parquet -->
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
 </project>
diff --git a/packaging/hudi-hive-bundle/pom.xml b/packaging/hudi-hive-bundle/pom.xml
index 72f00db..7ad1047 100644
--- a/packaging/hudi-hive-bundle/pom.xml
+++ b/packaging/hudi-hive-bundle/pom.xml
@@ -23,135 +23,16 @@
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
-
   <artifactId>hudi-hive-bundle</artifactId>
   <packaging>jar</packaging>
 
-  <dependencies>
-    <!-- Hoodie -->
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-hadoop-mr-bundle</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-hive</artifactId>
-      <version>${project.version}</version>
-      <exclusions>
-        <exclusion>
-          <!-- All other hoodie deps will come from hoodie-hadoop-mr-bundle -->
-          <groupId>org.apache.hudi</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <!-- Logging -->
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-    </dependency>
-
-    <!-- Thrift -->
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-      <version>${thrift.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libfb303</artifactId>
-      <version>0.9.3</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-dbcp</groupId>
-      <artifactId>commons-dbcp</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-
-    <!-- Httpcomponents -->
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpcore</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpclient</artifactId>
-    </dependency>
-
-    <!-- Hadoop -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-auth</artifactId>
-    </dependency>
-
-    <!-- Hive -->
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-service</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-jdbc</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-metastore</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-common</artifactId>
-    </dependency>
-  </dependencies>
+  <properties>
+    <checkstyle.skip>true</checkstyle.skip>
+    <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
+    <notice.file>HUDI_NOTICE.txt</notice.file>
+  </properties>
 
   <build>
-    <resources>
-       <resource>
-         <directory>src/main/resources</directory>
-       </resource>
-     </resources>
     <plugins>
       <plugin>
         <groupId>org.apache.rat</groupId>
@@ -160,7 +41,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
-        <version>2.4</version>
+        <version>${maven-shade-plugin.version}</version>
         <executions>
           <execution>
             <phase>package</phase>
@@ -169,53 +50,22 @@
             </goals>
             <configuration>
               <createSourcesJar>true</createSourcesJar>
-              <relocations>
-                 <relocation>
-                  <pattern>com.beust.</pattern>
-                  <shadedPattern>org.apache.hudi.com.beust.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                  <pattern>org.joda.</pattern>
-                  <shadedPattern>org.apache.hudi.org.joda.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                  <pattern>com.google.</pattern>
-                  <shadedPattern>org.apache.hudi.com.google.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.slf4j.</pattern>
-                   <shadedPattern>org.apache.hudi.org.slf4j.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.apache.commons.</pattern>
-                   <shadedPattern>org.apache.hudi.org.apache.commons.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.column</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.column</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.format.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.format.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.hadoop.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.hadoop.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.schema.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.schema.</shadedPattern>
-                 </relocation>
-              </relocations>
-              <createDependencyReducedPom>false</createDependencyReducedPom>
+              <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml
+              </dependencyReducedPomLocation>
               <artifactSet>
-                <excludes>
-                  <exclude>log4j:log4j</exclude>
-                  <exclude>org.apache.hadoop:*</exclude>
-                  <exclude>org.apache.hive:*</exclude>
-                  <exclude>org.apache.derby:derby</exclude>
-                </excludes>
+                <includes>
+                  <include>org.apache.hudi:hudi-common</include>
+                  <include>org.apache.hudi:hudi-hadoop-mr</include>
+                  <include>org.apache.hudi:hudi-hive</include>
+
+                  <include>com.beust:jcommander</include>
+                  <include>org.apache.parquet:parquet-avro</include>
+                  <include>com.esotericsoftware:kryo-shaded</include>
+                  <include>org.objenesis:objenesis</include>
+                  <include>com.esotericsoftware:minlog</include>
+                </includes>
               </artifactSet>
+              <createDependencyReducedPom>false</createDependencyReducedPom>
               <filters>
                 <filter>
                   <artifact>*:*</artifact>
@@ -235,11 +85,32 @@
         </executions>
       </plugin>
     </plugins>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+      </resource>
+      <resource>
+        <directory>src/test/resources</directory>
+      </resource>
+    </resources>
   </build>
 
-  <properties>
-    <checkstyle.skip>true</checkstyle.skip>
-    <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
-    <notice.file>HUDI_NOTICE.txt</notice.file>
-  </properties>
+  <dependencies>
+    <!-- Hoodie -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hadoop-mr-bundle</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hive</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
 </project>
diff --git a/packaging/hudi-presto-bundle/pom.xml b/packaging/hudi-presto-bundle/pom.xml
index fbaba74..17c3d2e 100644
--- a/packaging/hudi-presto-bundle/pom.xml
+++ b/packaging/hudi-presto-bundle/pom.xml
@@ -23,100 +23,16 @@
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
-
   <artifactId>hudi-presto-bundle</artifactId>
   <packaging>jar</packaging>
 
-  <dependencies>
-    <!-- Hoodie -->
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-hadoop-mr-bundle</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <!-- Logging -->
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-log4j12</artifactId>
-    </dependency>
-
-    <!-- Thrift -->
-    <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
-      <version>${thrift.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>joda-time</groupId>
-      <artifactId>joda-time</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-    </dependency>
-
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-dbcp</groupId>
-      <artifactId>commons-dbcp</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-io</groupId>
-      <artifactId>commons-io</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-
-    <!-- Httpcomponents-->
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpcore</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpclient</artifactId>
-    </dependency>
-
-    <!-- Hadoop -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-auth</artifactId>
-    </dependency>
-  </dependencies>
+  <properties>
+    <checkstyle.skip>true</checkstyle.skip>
+    <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
+    <notice.file>HUDI_NOTICE.txt</notice.file>
+  </properties>
 
   <build>
-    <resources>
-      <resource>
-        <directory>src/main/resources</directory>
-      </resource>
-    </resources>
     <plugins>
       <plugin>
         <groupId>org.apache.rat</groupId>
@@ -125,7 +41,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
-        <version>2.4</version>
+        <version>${maven-shade-plugin.version}</version>
         <executions>
           <execution>
             <phase>package</phase>
@@ -134,73 +50,35 @@
             </goals>
             <configuration>
               <createSourcesJar>true</createSourcesJar>
+              <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml
+              </dependencyReducedPomLocation>
+              <artifactSet>
+                <includes>
+                  <include>org.apache.hudi:hudi-common</include>
+                  <include>org.apache.hudi:hudi-hadoop-mr</include>
+
+                  <include>org.apache.parquet:parquet-avro</include>
+                  <include>com.esotericsoftware:kryo-shaded</include>
+                  <include>org.objenesis:objenesis</include>
+                  <include>com.esotericsoftware:minlog</include>
+                </includes>
+              </artifactSet>
               <relocations>
-                 <relocation>
-                  <pattern>com.beust.</pattern>
-                  <shadedPattern>org.apache.hudi.com.beust.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                  <pattern>org.joda.</pattern>
-                  <shadedPattern>org.apache.hudi.org.joda.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                  <pattern>com.google.</pattern>
-                  <shadedPattern>org.apache.hudi.com.google.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.slf4j.</pattern>
-                   <shadedPattern>org.apache.hudi.org.slf4j.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.apache.commons.</pattern>
-                   <shadedPattern>org.apache.hudi.org.apache.commons.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.column</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.column</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.format.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.format.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.hadoop.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.hadoop.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>parquet.schema.</pattern>
-                   <shadedPattern>org.apache.hudi.parquet.schema.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>com.esotericsoftware.kryo.</pattern>
-                   <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>org.objenesis.</pattern>
-                   <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
-                 </relocation>
-                 <relocation>
-                   <pattern>com.esotericsoftware.minlog.</pattern>
-                   <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
-                 </relocation>
+
+                <relocation>
+                  <pattern>com.esotericsoftware.kryo.</pattern>
+                  <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.objenesis.</pattern>
+                  <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.esotericsoftware.minlog.</pattern>
+                  <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
+                </relocation>
               </relocations>
               <createDependencyReducedPom>false</createDependencyReducedPom>
-              <artifactSet>
-                <excludes>
-                  <exclude>log4j:log4j</exclude>
-                  <exclude>org.apache.hadoop:*</exclude>
-                  <exclude>org.apache.hive:*</exclude>
-                  <exclude>org.apache.derby:derby</exclude>
-                  <!--Already a dependency in presto-hive connector-->
-                  <exclude>org.apache.thrift:*</exclude>
-                  <!--Provided by aws-java-sdk-core dependency in presto-hive connector-->
-                  <exclude>org.apache.httpcomponents:*</exclude>
-                  <!--Provided by hive-hadoop2-->
-                  <exclude>com.fasterxml.jackson.core:*</exclude>
-                  <exclude>com.fasterxml.jackson.datatype:jackson-datatype-guava</exclude>
-                  <exclude>org.apache.parquet:*</exclude>
-                </excludes>
-              </artifactSet>
               <filters>
                 <filter>
                   <artifact>*:*</artifact>
@@ -220,11 +98,27 @@
         </executions>
       </plugin>
     </plugins>
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+      </resource>
+      <resource>
+        <directory>src/test/resources</directory>
+      </resource>
+    </resources>
   </build>
 
-  <properties>
-    <checkstyle.skip>true</checkstyle.skip>
-    <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
-    <notice.file>HUDI_NOTICE.txt</notice.file>
-  </properties>
+  <dependencies>
+    <!-- Hoodie -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hadoop-mr-bundle</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
 </project>
diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml
index bcd5bea..d36d3a6 100644
--- a/packaging/hudi-spark-bundle/pom.xml
+++ b/packaging/hudi-spark-bundle/pom.xml
@@ -23,53 +23,25 @@
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>
-
-  <groupId>org.apache.hudi</groupId>
   <artifactId>hudi-spark-bundle</artifactId>
   <packaging>jar</packaging>
 
   <properties>
-    <log4j.version>1.2.17</log4j.version>
-    <junit.version>4.10</junit.version>
     <checkstyle.skip>true</checkstyle.skip>
     <notice.dir>${project.basedir}/src/main/resources/META-INF</notice.dir>
     <notice.file>HUDI_NOTICE.txt</notice.file>
   </properties>
 
   <build>
-     <resources>
-       <resource>
-         <directory>src/main/resources</directory>
-       </resource>
-     </resources>
      <plugins>
       <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>copy-dependencies</id>
-            <phase>prepare-package</phase>
-            <goals>
-              <goal>copy-dependencies</goal>
-            </goals>
-            <configuration>
-              <outputDirectory>${project.build.directory}/lib</outputDirectory>
-              <overWriteReleases>true</overWriteReleases>
-              <overWriteSnapshots>true</overWriteSnapshots>
-              <overWriteIfNewer>true</overWriteIfNewer>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
-        <version>2.4</version>
+        <version>${maven-shade-plugin.version}</version>
         <executions>
           <execution>
             <phase>package</phase>
@@ -78,54 +50,82 @@
             </goals>
             <configuration>
               <createSourcesJar>true</createSourcesJar>
+              <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml
+              </dependencyReducedPomLocation>
+              <artifactSet>
+                <includes>
+                  <include>org.apache.hudi:hudi-common</include>
+                  <include>org.apache.hudi:hudi-client</include>
+                  <include>org.apache.hudi:hudi-spark</include>
+                  <include>org.apache.hudi:hudi-hive</include>
+                  <include>org.apache.hudi:hudi-hadoop-mr</include>
+                  <include>org.apache.hudi:hudi-timeline-service</include>
+
+                  <include>com.beust:jcommander</include>
+                  <include>io.javalin:javalin</include>
+                  <!-- Spark only has mortbay jetty -->
+                  <include>org.eclipse.jetty:*</include>
+                  <include>org.eclipse.jetty.websocket:*</include>
+                  <include>org.jetbrains.kotlin:*</include>
+                  <include>org.rocksdb:rocksdbjni</include>
+                  <include>org.apache.httpcomponents:httpclient</include>
+                  <include>org.apache.httpcomponents:fluent-hc</include>
+                  <include>org.antlr:stringtemplate</include>
+                  <include>org.apache.parquet:parquet-avro</include>
+
+                  <include>com.twitter:bijection-avro_2.11</include>
+                  <include>com.twitter:bijection-core_2.11</include>
+                  <include>io.dropwizard.metrics:metrics-core</include>
+                  <include>io.dropwizard.metrics:metrics-graphite</include>
+                  <include>com.yammer.metrics:metrics-core</include>
+
+                  <include>org.apache.hive:hive-common</include>
+                  <include>org.apache.hive:hive-service</include>
+                  <include>org.apache.hive:hive-service-rpc</include>
+                  <include>org.apache.hive:hive-metastore</include>
+                  <include>org.apache.hive:hive-jdbc</include>
+                </includes>
+              </artifactSet>
               <relocations>
                 <relocation>
-                  <pattern>com.beust.</pattern>
-                  <shadedPattern>org.apache.hudi.com.beust.</shadedPattern>
+                  <pattern>com.beust.jcommander.</pattern>
+                  <shadedPattern>org.apache.hudi.com.beust.jcommander.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>org.joda.</pattern>
-                  <shadedPattern>org.apache.hudi.org.joda.</shadedPattern>
+                  <pattern>org.apache.hive.jdbc.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.jdbc.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>com.google.</pattern>
-                  <shadedPattern>org.apache.hudi.com.google.</shadedPattern>
+                  <pattern>org.apache.hadoop.hive.metastore.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.metastore.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>org.slf4j.</pattern>
-                  <shadedPattern>org.apache.hudi.org.slf4j.</shadedPattern>
+                  <pattern>org.apache.hive.common.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.common.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>org.apache.</pattern>
-                  <shadedPattern>org.apache.hudi.org.apache.</shadedPattern>
-                  <excludes>
-                    <exclude>com.databricks.spark.**</exclude>
-                    <exclude>org.apache.avro.**</exclude>
-                    <exclude>org.apache.derby.**</exclude>
-                    <exclude>org.apache.hadoop.**</exclude>
-                    <exclude>org.apache.hive.**</exclude>
-                    <exclude>org.apache.hudi.**</exclude>
-                    <exclude>org.apache.logging.log4j.**</exclude>
-                    <exclude>org.apache.log4j.**</exclude>
-                    <exclude>org.apache.spark.**</exclude>
-                    <exclude>org.apache.thrift.**</exclude>
-                  </excludes>
+                  <pattern>org.apache.hadoop.hive.common.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.common.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>parquet.column</pattern>
-                  <shadedPattern>org.apache.hudi.parquet.column</shadedPattern>
+                  <pattern>org.apache.hadoop.hive.conf.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.conf.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>parquet.format.</pattern>
-                  <shadedPattern>org.apache.hudi.parquet.format.</shadedPattern>
+                  <pattern>org.apache.hive.service.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.service.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.service.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.service.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>parquet.hadoop.</pattern>
-                  <shadedPattern>org.apache.hudi.parquet.hadoop.</shadedPattern>
+                  <pattern>com.codahale.metrics.</pattern>
+                  <shadedPattern>org.apache.hudi.com.codahale.metrics.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>parquet.schema</pattern>
-                  <shadedPattern>org.apache.hudi.parquet.schema</shadedPattern>
+                  <pattern>org.apache.commons.codec.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.commons.codec.</shadedPattern>
                 </relocation>
                 <!-- TODO: Revisit GH ISSUE #533 & PR#633-->
                 <!--
@@ -162,35 +162,8 @@
                   <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.serde2.</shadedPattern>
                 </relocation>
                 -->
-                <relocation>
-                  <pattern>com.esotericsoftware.kryo.</pattern>
-                  <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>org.objenesis.</pattern>
-                  <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>com.esotericsoftware.minlog.</pattern>
-                  <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
-                </relocation>
               </relocations>
-              <createDependencyReducedPom>false</createDependencyReducedPom>
-              <artifactSet>
-                <excludes>
-                  <exclude>com.databricks:spark-avro_2.11</exclude>
-                  <exclude>log4j:*</exclude>
-                  <exclude>org.apache.avro:*</exclude>
-                  <exclude>org.apache.derby:derby</exclude>
-                  <exclude>org.apache.hadoop:*</exclude>
-                  <exclude>org.apache.hbase:*</exclude>
-                  <!-- Just include hive-common, hive-serde, hive-service, hive-metastore and hive-jdbc -->
-                  <exclude>org.apache.hive:hive-exec</exclude>
-                  <exclude>org.apache.hive:hive-shims</exclude>
-                  <exclude>org.apache.spark:*</exclude>
-                </excludes>
-              </artifactSet>
-              <filters>
+             <filters>
                 <filter>
                   <artifact>*:*</artifact>
                   <excludes>
@@ -209,25 +182,26 @@
         </executions>
       </plugin>
     </plugins>
+     <resources>
+       <resource>
+         <directory>src/main/resources</directory>
+       </resource>
+       <resource>
+         <directory>src/test/resources</directory>
+       </resource>
+     </resources>
   </build>
 
   <dependencies>
-    <!-- Scala -->
-    <dependency>
-      <groupId>org.scala-lang</groupId>
-      <artifactId>scala-library</artifactId>
-      <version>${scala.version}</version>
-    </dependency>
-
     <!-- Hoodie -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-common</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
+      <artifactId>hudi-client</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -246,117 +220,42 @@
       <version>${project.version}</version>
     </dependency>
 
-    <!-- Logging -->
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-      <version>${log4j.version}</version>
-    </dependency>
-
-    <!-- Fasterxml -->
-    <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
-
-    <!-- Avro -->
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-    </dependency>
-
-    <!-- Spark -->
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_2.11</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_2.11</artifactId>
-    </dependency>
-
-    <!-- Spark (Packages) -->
-    <dependency>
-      <groupId>com.databricks</groupId>
-      <artifactId>spark-avro_2.11</artifactId>
-      <version>4.0.0</version>
-    </dependency>
-
+    <!-- Parquet -->
     <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-codec</groupId>
-      <artifactId>commons-codec</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-dbcp</groupId>
-      <artifactId>commons-dbcp</artifactId>
-    </dependency>
-
-    <!-- Hadoop -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <scope>provided</scope>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+      <scope>compile</scope>
     </dependency>
 
-    <!-- Hive - Compile -->
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-jdbc</artifactId>
-      <version>${hive.version}</version>
-      <classifier>standalone</classifier>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <!-- TODO: Reinvestigate PR 633 -->
+    <!-- Hive -->
     <dependency>
       <groupId>${hive.groupid}</groupId>
       <artifactId>hive-service</artifactId>
       <version>${hive.version}</version>
       <scope>compile</scope>
     </dependency>
+
     <dependency>
       <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-jdbc</artifactId>
+      <artifactId>hive-service-rpc</artifactId>
       <version>${hive.version}</version>
       <scope>compile</scope>
     </dependency>
+
     <dependency>
       <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-serde</artifactId>
+      <artifactId>hive-jdbc</artifactId>
       <version>${hive.version}</version>
       <scope>compile</scope>
     </dependency>
+
     <dependency>
       <groupId>${hive.groupid}</groupId>
       <artifactId>hive-metastore</artifactId>
       <version>${hive.version}</version>
       <scope>compile</scope>
     </dependency>
+
     <dependency>
       <groupId>${hive.groupid}</groupId>
       <artifactId>hive-common</artifactId>
@@ -364,13 +263,6 @@
       <scope>compile</scope>
     </dependency>
     <!-- TODO: Reinvestigate PR 633 -->
-
-    <dependency>
-      <groupId>org.scalatest</groupId>
-      <artifactId>scalatest_2.11</artifactId>
-      <version>${scalatest.version}</version>
-      <scope>test</scope>
-    </dependency>
   </dependencies>
 </project>
 
diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml
index e002c11..bf992da 100644
--- a/packaging/hudi-utilities-bundle/pom.xml
+++ b/packaging/hudi-utilities-bundle/pom.xml
@@ -36,21 +36,13 @@
   <build>
     <plugins>
       <plugin>
-        <groupId>org.jacoco</groupId>
-        <artifactId>jacoco-maven-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-compiler-plugin</artifactId>
-        <configuration>
-          <source>1.8</source>
-          <target>1.8</target>
-        </configuration>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-shade-plugin</artifactId>
-        <version>3.1.1</version>
+        <version>${maven-shade-plugin.version}</version>
         <executions>
           <execution>
             <phase>package</phase>
@@ -63,10 +55,6 @@
               </dependencyReducedPomLocation>
               <artifactSet>
                 <includes>
-                  <include>commons-codec:commons-codec</include>
-                  <include>commons-dbcp:commons-dbcp</include>
-                  <include>commons-lang:commons-lang</include>
-                  <include>commons-pool:commons-pool</include>
                   <include>org.apache.hudi:hudi-common</include>
                   <include>org.apache.hudi:hudi-client</include>
                   <include>org.apache.hudi:hudi-utilities</include>
@@ -74,41 +62,38 @@
                   <include>org.apache.hudi:hudi-hive</include>
                   <include>org.apache.hudi:hudi-hadoop-mr</include>
                   <include>org.apache.hudi:hudi-timeline-service</include>
-                  <include>com.beust:jcommander</include>
 
+                  <include>com.beust:jcommander</include>
                   <include>io.javalin:javalin</include>
-                  <include>org.jetbrains.kotlin:*</include>
+                  <!-- Spark only has mortbay jetty -->
                   <include>org.eclipse.jetty:*</include>
                   <include>org.eclipse.jetty.websocket:*</include>
+                  <include>org.jetbrains.kotlin:*</include>
                   <include>org.rocksdb:rocksdbjni</include>
                   <include>org.apache.httpcomponents:httpclient</include>
                   <include>org.apache.httpcomponents:fluent-hc</include>
                   <include>org.antlr:stringtemplate</include>
-                  <include>commons-io:commons-io</include>
-                  <include>commons-logging:commons-logging</include>
                   <include>org.apache.parquet:parquet-avro</include>
 
                   <include>com.twitter:bijection-avro_2.11</include>
                   <include>com.twitter:bijection-core_2.11</include>
-                  <include>org.apache.parquet:parquet-avro</include>
                   <include>io.confluent:kafka-avro-serializer</include>
                   <include>io.confluent:common-config</include>
                   <include>io.confluent:common-utils</include>
                   <include>io.confluent:kafka-schema-registry-client</include>
                   <include>io.dropwizard.metrics:metrics-core</include>
                   <include>io.dropwizard.metrics:metrics-graphite</include>
+                  <include>com.yammer.metrics:metrics-core</include>
                   <include>org.apache.spark:spark-streaming-kafka-0-8_2.11</include>
                   <include>org.apache.kafka:kafka_2.11</include>
                   <include>com.101tec:zkclient</include>
                   <include>org.apache.kafka:kafka-clients</include>
+
                   <include>org.apache.hive:hive-common</include>
                   <include>org.apache.hive:hive-service</include>
+                  <include>org.apache.hive:hive-service-rpc</include>
                   <include>org.apache.hive:hive-metastore</include>
                   <include>org.apache.hive:hive-jdbc</include>
-                  <include>com.esotericsoftware:kryo-shaded</include>
-                  <include>org.objenesis:objenesis</include>
-                  <include>com.esotericsoftware:minlog</include>
-                  <include>com.yammer.metrics:metrics-core</include>
                 </includes>
               </artifactSet>
               <relocations>
@@ -117,18 +102,6 @@
                   <shadedPattern>org.apache.hudi.com.beust.jcommander.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>org.apache.commons.dbcp.</pattern>
-                  <shadedPattern>org.apache.hudi.org.apache.commons.dbcp.</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>org.apache.commons.lang.</pattern>
-                  <shadedPattern>org.apache.hudi.org.apache.commons.lang.</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>org.apache.commons.pool.</pattern>
-                  <shadedPattern>org.apache.hudi.org.apache.commons.pool.</shadedPattern>
-                </relocation>
-                <relocation>
                   <pattern>org.apache.hive.jdbc.</pattern>
                   <shadedPattern>org.apache.hudi.org.apache.hive.jdbc.</shadedPattern>
                 </relocation>
@@ -157,18 +130,6 @@
                   <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.service.</shadedPattern>
                 </relocation>
                 <relocation>
-                  <pattern>com.esotericsoftware.kryo.</pattern>
-                  <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>org.objenesis.</pattern>
-                  <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
-                </relocation>
-                <relocation>
-                  <pattern>com.esotericsoftware.minlog.</pattern>
-                  <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
-                </relocation>
-                <relocation>
                   <pattern>com.codahale.metrics.</pattern>
                   <shadedPattern>org.apache.hudi.com.codahale.metrics.</shadedPattern>
                 </relocation>
@@ -217,12 +178,12 @@
     <!-- Hoodie -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-common</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
+      <artifactId>hudi-client</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -247,231 +208,47 @@
       <version>${project.version}</version>
     </dependency>
 
-    <!-- Logging -->
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
-    </dependency>
-
-    <!-- Fasterxml -->
-    <dependency>
-      <groupId>com.fasterxml.jackson.module</groupId>
-      <artifactId>jackson-module-scala_2.11</artifactId>
-    </dependency>
-
-    <!-- Avro -->
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-mapred</artifactId>
-    </dependency>
-
     <!-- Parquet -->
     <dependency>
       <groupId>org.apache.parquet</groupId>
       <artifactId>parquet-avro</artifactId>
-    </dependency>
-
-    <!-- Spark -->
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_2.11</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_2.11</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_2.11</artifactId>
-      <version>${spark.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming-kafka-0-8_2.11</artifactId>
-      <version>${spark.version}</version>
-    </dependency>
-
-    <!-- Dropwizard Metrics -->
-    <dependency>
-      <groupId>io.dropwizard.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>io.javalin</groupId>
-      <artifactId>javalin</artifactId>
-      <version>2.4.0</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.yammer.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-      <version>2.2.0</version>
-    </dependency>
-
-    <!-- Used for SQL templating -->
-    <dependency>
-      <groupId>org.antlr</groupId>
-      <artifactId>stringtemplate</artifactId>
-      <version>4.0.2</version>
-    </dependency>
-
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>bijection-avro_2.11</artifactId>
-      <version>0.9.2</version>
-    </dependency>
-
-    <!-- Kafka -->
-    <dependency>
-      <groupId>io.confluent</groupId>
-      <artifactId>kafka-avro-serializer</artifactId>
-      <version>3.0.0</version>
-    </dependency>
-    <dependency>
-      <groupId>io.confluent</groupId>
-      <artifactId>common-config</artifactId>
-      <version>3.0.0</version>
-    </dependency>
-    <dependency>
-      <groupId>io.confluent</groupId>
-      <artifactId>common-utils</artifactId>
-      <version>3.0.0</version>
-    </dependency>
-    <dependency>
-      <groupId>io.confluent</groupId>
-      <artifactId>kafka-schema-registry-client</artifactId>
-      <version>3.0.0</version>
-    </dependency>
-
-    <!-- Apache Commons -->
-    <dependency>
-      <groupId>commons-codec</groupId>
-      <artifactId>commons-codec</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-dbcp</groupId>
-      <artifactId>commons-dbcp</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>commons-pool</groupId>
-      <artifactId>commons-pool</artifactId>
-    </dependency>
-
-    <!-- Httpcomponents -->
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpcore</artifactId>
-    </dependency>
-
-    <!-- Hadoop -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-common</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
-        </exclusion>
-      </exclusions>
+      <scope>compile</scope>
     </dependency>
 
     <!-- Hive -->
     <dependency>
       <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-jdbc</artifactId>
+      <artifactId>hive-service</artifactId>
       <version>${hive.version}</version>
-      <classifier>standalone</classifier>
-      <exclusions>
-        <exclusion>
-          <groupId>org.slf4j</groupId>
-          <artifactId>slf4j-api</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>servlet-api</artifactId>
-        </exclusion>
-      </exclusions>
+      <scope>compile</scope>
     </dependency>
 
-    <!-- Hoodie - Test -->
     <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
-      <version>${project.version}</version>
-      <classifier>tests</classifier>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
-      <version>${project.version}</version>
-      <classifier>tests</classifier>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-hive</artifactId>
-      <version>${project.version}</version>
-      <classifier>tests</classifier>
-      <type>test-jar</type>
-      <scope>test</scope>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-service-rpc</artifactId>
+      <version>${hive.version}</version>
+      <scope>compile</scope>
     </dependency>
 
-    <!-- Hadoop - Test -->
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <classifier>tests</classifier>
-      <exclusions>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet.jsp</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <classifier>tests</classifier>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-jdbc</artifactId>
+      <version>${hive.version}</version>
+      <scope>compile</scope>
     </dependency>
 
-    <!-- Hive - Test -->
     <dependency>
       <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-exec</artifactId>
+      <artifactId>hive-metastore</artifactId>
       <version>${hive.version}</version>
-      <scope>test</scope>
+      <scope>compile</scope>
     </dependency>
 
     <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-common</artifactId>
+      <version>${hive.version}</version>
+      <scope>compile</scope>
     </dependency>
   </dependencies>
 </project>
diff --git a/pom.xml b/pom.xml
index a4ef40f..894667a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -131,6 +131,8 @@
   <properties>
     <maven-jar-plugin.version>2.6</maven-jar-plugin.version>
     <maven-surefire-plugin.version>2.19.1</maven-surefire-plugin.version>
+    <maven-shade-plugin.version>3.1.1</maven-shade-plugin.version>
+
     <fasterxml.version>2.6.7</fasterxml.version>
     <glassfish.version>2.17</glassfish.version>
     <parquet.version>1.8.1</parquet.version>
@@ -164,25 +166,12 @@
   </properties>
 
   <scm>
-    <connection>scm:git:git@github.com:uber/hudi.git</connection>
-    <developerConnection>scm:git:git@github.com:uber/hudi.git</developerConnection>
-    <url>git@github.com:uber/hudi.git</url>
+    <connection>scm:git:git@github.com:apache/incubator-hudi.git</connection>
+    <developerConnection>scm:git:git@github.com:apache/incubator-hudi.git</developerConnection>
+    <url>git@github.com:apache/incubator-hudi.git</url>
     <tag>HEAD</tag>
   </scm>
 
-  <mailingLists>
-    <mailingList>
-      <name>User List</name>
-      <post>hudi-user@googlegroups.com</post>
-      <archive>https://groups.google.com/d/forum/hudi-user/</archive>
-    </mailingList>
-    <mailingList>
-      <name>Developer List</name>
-      <post>hudi-dev@googlegroups.com</post>
-      <archive>https://groups.google.com/d/forum/hudi-dev/</archive>
-    </mailingList>
-  </mailingLists>
-
   <build>
     <plugins>
       <plugin>
@@ -459,16 +448,6 @@
         <artifactId>log4j</artifactId>
         <version>${log4j.version}</version>
       </dependency>
-      <dependency>
-        <groupId>org.slf4j</groupId>
-        <artifactId>slf4j-api</artifactId>
-        <version>${slf4j.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.slf4j</groupId>
-        <artifactId>slf4j-log4j12</artifactId>
-        <version>${slf4j.version}</version>
-      </dependency>
 
       <!-- Fasterxml -->
       <dependency>
@@ -526,11 +505,7 @@
         <groupId>org.apache.avro</groupId>
         <artifactId>avro</artifactId>
         <version>${avro.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.avro</groupId>
-        <artifactId>avro-mapred</artifactId>
-        <version>${avro.version}</version>
+        <scope>provided</scope>
       </dependency>
 
       <!-- Parquet -->
@@ -538,6 +513,7 @@
         <groupId>org.apache.parquet</groupId>
         <artifactId>parquet-avro</artifactId>
         <version>${parquet.version}</version>
+        <scope>provided</scope>
       </dependency>
 
       <!-- Spark -->
@@ -610,38 +586,6 @@
         <version>5.17.2</version>
       </dependency>
 
-      <!-- Apache Commons -->
-      <dependency>
-        <groupId>commons-codec</groupId>
-        <artifactId>commons-codec</artifactId>
-        <version>1.4</version>
-      </dependency>
-      <dependency>
-        <groupId>commons-io</groupId>
-        <artifactId>commons-io</artifactId>
-        <version>2.6</version>
-      </dependency>
-      <dependency>
-        <groupId>commons-lang</groupId>
-        <artifactId>commons-lang</artifactId>
-        <version>2.6</version>
-      </dependency>
-      <dependency>
-        <groupId>commons-logging</groupId>
-        <artifactId>commons-logging</artifactId>
-        <version>1.2</version>
-      </dependency>
-      <dependency>
-        <groupId>commons-dbcp</groupId>
-        <artifactId>commons-dbcp</artifactId>
-        <version>1.4</version>
-      </dependency>
-      <dependency>
-        <groupId>commons-pool</groupId>
-        <artifactId>commons-pool</artifactId>
-        <version>1.4</version>
-      </dependency>
-
       <!-- Httpcomponents -->
       <dependency>
         <groupId>org.apache.httpcomponents</groupId>
diff --git a/style/checkstyle.xml b/style/checkstyle.xml
index 268fefc..947d141 100644
--- a/style/checkstyle.xml
+++ b/style/checkstyle.xml
@@ -259,6 +259,7 @@
         </module>
         <module name="CommentsIndentation"/>
         <module name="IllegalImport">
+            <property name="illegalPkgs" value="org.apache.commons" />
             <property name="illegalClasses" value="java.util.Optional, com.google.common.base.Optional" />
         </module>
     </module>