1
0

[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
This commit is contained in:
vinoth chandar
2019-09-02 16:15:55 -07:00
committed by Balaji Varadarajan
parent 0e6f078ec4
commit 7a973a6944
60 changed files with 689 additions and 1380 deletions

View File

@@ -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;

View File

@@ -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

View File

@@ -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;

View File

@@ -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;

View File

@@ -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;

View File

@@ -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;

View File

@@ -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,

View File

@@ -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;